You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/01/31 21:48:25 UTC

[GitHub] [druid] jon-wei opened a new pull request #9301: Join filter pushdown initial implementation

jon-wei opened a new pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301
 
 
   This PR adds an initial implementation for pushing down filters to the base table in a join query.
   
   The bulk of logic added is in the new JoinFilterAnalyzer utils class. This class has methods that split a Filter into a base table portion that can be pushed down and a post-join portion. The analyzer is capable of rewriting filters that apply to columns on the join tables into filters on the base table.
   
   Current limitations:
   - Only selector filters can be rewritten
   - IS NULL filters are not pushed down
   - There is currently no limit on the IN filter size for rewrites, future work can improve the heuristics for deciding when to rewrite filters
   - The filter push down could benefit from more efficient lookups for values of non-key columns in Joinables
   
   This PR has:
   - [x] been self-reviewed.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   
   ##### Key changed/added classes in this PR
    * `JoinFilterAnalyzer`
    * `HashJoinSegmentStorageAdapter`
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376061258
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
 
 Review comment:
   Rad.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375668999
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
 
 Review comment:
   This is for cases like the fact->region->country joins in the tests, where the three tables are all joined on `countryIsoCode`, but the last table is joined to the second table, instead of the base table, and we have a filter on a column from the last table in the chain. 
   
   `JoinFilterAnalyzerTest.test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCountryName` is an example where this is used.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374997493
 
 

 ##########
 File path: processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
 ##########
 @@ -1288,103 +1313,1163 @@ public void test_makeCursors_errorOnNonKeyBasedJoin()
     );
   }
 
-  private JoinableClause factToCountryNameUsingIsoCodeLookup(final JoinType joinType)
+  // Filter push down tests
 
 Review comment:
   I split out the segment setup and some helper methods out of `HashJoinSegmentStorageAdapterTest` into `BaseHashJoinSegmentStorageAdapterTest` since the JoinFilterAnalyzer needs a HashJoinSegmentStorageAdapter, both `JoinFilterAnalyzerTest` and `HashJoinSegmentStorageAdapterTest` now extend the base.
   
   The filter push down tests have been moved into `JoinFilterAnalyzerTest`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376041101
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
 ##########
 @@ -62,6 +64,23 @@ public ExpressionVirtualColumn(
     this.parsedExpression = Suppliers.memoize(() -> Parser.parse(expression, macroTable));
   }
 
+  /**
+   * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression.
+   */
+  public ExpressionVirtualColumn(
+      String name,
+      Expr parsedExpression,
+      ValueType outputType
+  )
+  {
+    this.name = Preconditions.checkNotNull(name, "name");
+    // Unfortunately this string representation can't be reparsed into the same expression, might be useful
+    // if the expression system supported that
+    this.expression = parsedExpression.toString();
 
 Review comment:
   I think this is a reasonable course of action. Could you please also raise a followup github issue.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666271
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
 
 Review comment:
   Right now it could be changed to just accept a list of joinable clauses, but in the future I think the filter analysis would likely need to check the column capabilities and other info from the adapter, so I think it should accept the adapter instead

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375020209
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
 
 Review comment:
   It looks like this is the only spot getPushdownVirtualColumns is null-guarded, and in this case, using an empty list instead of null would have the same effect. Maybe nix the nullability and use an empty list instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374998499
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
 
 Review comment:
   I just realized there's a bug with this part (it needs to create a globally unique virtual column name), will fix in a follow-on PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666869
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
 
 Review comment:
   Added a `getCorrelatedColumnValues` method on Joinable

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375009323
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
 
 Review comment:
   This code assumes that no two clauses have the same prefix, and (maybe also?) that the prefixes don't shadow each other. I don't think anything currently verifies either of those preconditions. It'd be good to add validation somewhere. Maybe right here, or maybe in HashJoinSegment's constructor. Maybe add a `Joinables` function that verifies it and call it wherever might care.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129982
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
 
 Review comment:
   I opened #9327 to track the query failure issue and #9328 to track the adjustments needed here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129098
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
 
 Review comment:
   I opened https://github.com/apache/druid/issues/9329 and linked it in comment here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374279951
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java
 ##########
 @@ -234,4 +235,23 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask)
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Do you want to add a unit test to `AndFilterTest` that uses `EqualsVerifier` (similar to https://github.com/apache/druid/blob/master/processing/src/test/java/org/apache/druid/query/QueryDataSourceTest.java#L142)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667689
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
 
 Review comment:
   Adjusted spelling to equiConditions

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129098
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
 
 Review comment:
   I opened https://github.com/apache/druid/issues/9329 and linked it in a comment here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667526
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
 
 Review comment:
   Hm, good point, I changed the map values to `Optional<List<JoinFilterColumnCorrelationAnalysis>>`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996553
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/InFilter.java
 ##########
 @@ -235,4 +236,26 @@ public DruidDoublePredicate makeDoublePredicate()
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Added an EqualsVerifier test

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375007275
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
 
 Review comment:
   It would be great to have a javadoc here describing what kind of analysis this class is trying to do, and why. Something like the javadocs at the top of JoinConditionAnalysis and DataSourceAnalysis.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666532
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
 
 Review comment:
   Hm, that sounds good, but can we address this in a follow-on PR since it's not specific to the filtering code?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666594
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
 
 Review comment:
   Adjusted the wording to the suggested

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375023041
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
 
 Review comment:
   `null` is used for two things here:
   
   (a) correlationCache hasn't been populated yet
   (b) correlationCache has been populated, but findCorrelatedBaseTableColumns returned null
   
   Would it make sense to have findCorrelatedBaseTableColumns return `Optional<List<JoinFilterColumnCorrelationAnalysis>>` rather than `@Nullable List<JoinFilterColumnCorrelationAnalysis>`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375019767
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
 
 Review comment:
   nit: Inconsistent spelling of "pushdown" (or "pushDown"?) between isCanPushDown, getPushdownFilter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375026182
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
 
 Review comment:
   small suggestion: `!requiredBindings.stream().allMatch( blah blah blah )` may be more readable

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375669997
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
 
 Review comment:
   Hm, it must have been a relic from an earlier version of the code, adjusted as suggested

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375010728
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
 
 Review comment:
   Maybe clearer to say that this single clause is expected to be either an OR or a leaf filter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374997063
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @param filter           Filter to be rewritten
+   * @param prefixes         Map of join table prefixes to clauses
+   * @param equiconditions   Map of equiconditions
+   * @param correlationCache Cache of column correlation analyses
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      Filter filter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    assert (filter instanceof SelectorFilter);
+    SelectorFilter selectorFilter = (SelectorFilter) filter;
+
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            filter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filter,
+        filter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
+          if (adapter.isBaseColumn(identifier)) {
+            terminate = true;
+            correlatedBaseColumns.add(findMappingFor);
+          }
+        }
+      }
+
+      if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) {
+        return null;
+      }
+
+      correlations.add(
+          new JoinFilterColumnCorrelationAnalysis(
+              rhsColumn,
+              correlatedBaseColumns,
+              correlatedBaseExpressions
+          )
+      );
+    }
+
+    return correlations;
+  }
+
+  private static boolean filterMatchesNull(Filter filter)
+  {
+    ValueMatcher valueMatcher = filter.makeMatcher(new AllNullColumnSelectorFactory());
 
 Review comment:
   Whoops, made a private static AllNullColumnSelectorFactory 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375024626
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
+          if (adapter.isBaseColumn(identifier)) {
+            terminate = true;
+            correlatedBaseColumns.add(findMappingFor);
+          }
+        }
+      }
+
+      if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) {
+        return null;
+      }
+
+      correlations.add(
+          new JoinFilterColumnCorrelationAnalysis(
+              rhsColumn,
+              correlatedBaseColumns,
+              correlatedBaseExpressions
+          )
+      );
+    }
+
+    return correlations;
+  }
+
+  private static boolean filterMatchesNull(Filter filter)
+  {
+    ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY);
+    return valueMatcher.matches();
+  }
+
+  private static class AllNullColumnSelectorFactory implements ColumnSelectorFactory
 
 Review comment:
   This class is pretty jam packed with different concepts and inner classes, maybe it would make sense to put it in its own `org.apache.druid.segment.join.filter` package and split it up into different classes? (Sort of like DataSourceAnalysis and its friend PreJoinableClause in `org.apache.druid.query.planning`, but this one is even more complex)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374319929
 
 

 ##########
 File path: processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
 ##########
 @@ -1288,103 +1313,1163 @@ public void test_makeCursors_errorOnNonKeyBasedJoin()
     );
   }
 
-  private JoinableClause factToCountryNameUsingIsoCodeLookup(final JoinType joinType)
+  // Filter push down tests
 
 Review comment:
   The test coverage of the filter push down code is great!
   
   What do you think about making many of these test cases as unit tests in `JoinFilterAnalyzerTest` instead? That way it's more straightforward to map the test case to the relevant code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374997535
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @param filter           Filter to be rewritten
+   * @param prefixes         Map of join table prefixes to clauses
+   * @param equiconditions   Map of equiconditions
+   * @param correlationCache Cache of column correlation analyses
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      Filter filter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    assert (filter instanceof SelectorFilter);
+    SelectorFilter selectorFilter = (SelectorFilter) filter;
 
 Review comment:
   Changed to SelectorFilter

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm merged pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm merged pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667304
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
 
 Review comment:
   Adjusted getPushdownVirtualColumns to use an empty list instead

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996670
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java
 ##########
 @@ -127,4 +128,35 @@ public String toString()
   {
     return StringUtils.format("%s = %s", dimension, value);
   }
+
+  public String getDimension()
+  {
+    return dimension;
+  }
+
+  public String getValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Added an EqualsVerifier test

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375025840
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
 
 Review comment:
   There could be more than one equi-condition for the same rhs column. It looks like this code disregards that possibility.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667062
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
 
 Review comment:
   I moved this into a separate file, the tests need to be able to create expected splits so the new class is public

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on issue #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#issuecomment-583074243
 
 
   @jon-wei do these TC inspections look legit?
   
   ![image](https://user-images.githubusercontent.com/1214075/73972123-1376ad00-48d5-11ea-957f-871a7a3aa49a.png)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375016811
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
 
 Review comment:
   Do people outside this class need to be able to make their own? Could be `private` if not.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376049916
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
 
 Review comment:
   Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376061041
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
 
 Review comment:
   I just read the javadoc. Could you please clarify (or rename) what "main column" means and also where "correlationColumnName" is supposed to come from? They're concepts that haven't been defined elsewhere in the class and it's not immediately clear to me what they mean. I'm guessing both columns are from the Joinable itself, and "main column" means the one being searched and "correlation column" means the one being retrieved.
   
   Currently it says:
   
   > Given a key column name and value, return all the values of the column denoted by correlationColumnName that appear in rows where the main column has the provided main column value.
   
   If my guesses are right then how about:
   
   > Searches a column from this Joinable for a particular value, finds rows that match and returns values of a second column for those rows.
   >
   > Arguments "searchColumn", "searchValue", "retrievalColumn"

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667427
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
 
 Review comment:
   Changed to the suggested

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996525
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java
 ##########
 @@ -306,4 +307,26 @@ private boolean doesMatch(String input)
     }
     return (lowerComparing >= 0) && (upperComparing >= 0);
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Added an EqualsVerifier test

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375030593
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
 
 Review comment:
   Is there a reason for this cast? Seems like we could let it go in to `newFilters` as an uncast Filter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667392
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
 
 Review comment:
   Added a `Filters.and` method and adjusted this and QueryableIndexStroageAdapter

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375025418
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
 
 Review comment:
   Two conditions can refer to the same rhs column. Should this be a Set?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375665780
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
 
 Review comment:
   Added javadoc here describing the goal and details about the push down and rewrites

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375020776
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
 
 Review comment:
   Small suggestion: might be nice to make this a helper method like `Filters.and(List<Filter>)`. This logic is duplicated in QueryableIndexStroageAdapter, so they could both use such a helper.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375691932
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
 
 Review comment:
   My original intent was related to the comment here about handling cases where one RHS column is joined to multiple columns: https://github.com/apache/druid/pull/9301#discussion_r375025840
   
   It's not implemented right now in `JoinFilterAnalyzer.findCorrelatedBaseTableColumns` (and should be), but in that case there would be multiple base columns correlated to the RHS filtering column.
   
   Since a query of that type doesn't run successfully right now, can we address this in a follow-on PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375013414
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause from a filter that is in CNF
+   * @param prefixes         Map of table prefixes
+   * @param equiconditions   Equicondition map
+   * @param correlationCache Cache of column correlation analyses
 
 Review comment:
   It looks like it's expected that callers will modify this parameter. It'd be good to note that (and any other case where it's expected that parameters will be modified).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374308360
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
 
 Review comment:
   Similar comment about testing with `EqualsVerifier`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376061804
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
 
 Review comment:
   Sure, but please raise a github issue and mention it in a comment.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376067428
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
 
 Review comment:
   Hmm, I see.
   
   Would you mind adding some comments to findCorrelatedBaseTableColumns explaining the idea behind the logic?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996640
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java
 ##########
 @@ -218,4 +219,23 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask)
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   There was no `OrFilterTest` class, I added one and an EqualsVerifier test

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667642
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
+          if (adapter.isBaseColumn(identifier)) {
+            terminate = true;
+            correlatedBaseColumns.add(findMappingFor);
+          }
+        }
+      }
+
+      if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) {
+        return null;
+      }
+
+      correlations.add(
+          new JoinFilterColumnCorrelationAnalysis(
+              rhsColumn,
+              correlatedBaseColumns,
+              correlatedBaseExpressions
+          )
+      );
+    }
+
+    return correlations;
+  }
+
+  private static boolean filterMatchesNull(Filter filter)
+  {
+    ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY);
+    return valueMatcher.matches();
+  }
+
+  private static class AllNullColumnSelectorFactory implements ColumnSelectorFactory
 
 Review comment:
   I split out the inner classes and JoinFilterAnalyzer into a new package `org.apache.druid.segment.join.filter`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129637
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
 
 Review comment:
   I opened https://github.com/apache/druid/issues/9327 to track the query failure issue and https://github.com/apache/druid/issues/9328 to track the adjustments needed here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375029851
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
 ##########
 @@ -62,6 +64,23 @@ public ExpressionVirtualColumn(
     this.parsedExpression = Suppliers.memoize(() -> Parser.parse(expression, macroTable));
   }
 
+  /**
+   * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression.
+   */
+  public ExpressionVirtualColumn(
+      String name,
+      Expr parsedExpression,
+      ValueType outputType
+  )
+  {
+    this.name = Preconditions.checkNotNull(name, "name");
+    // Unfortunately this string representation can't be reparsed into the same expression, might be useful
+    // if the expression system supported that
+    this.expression = parsedExpression.toString();
 
 Review comment:
   This code is nasty for the reason you mentioned in your comment. I don't think it'd be good to leave it like this, since it means any attempt to serialize this virtual column will yield something unusable, potentially leading to latent bugs.
   
   I'd almost consider _not_ supporting pushdown of filters where the lhs is an expression until we have a way to actually convert parsed expressions to strings.
   
   Another option:
   
   1) Make a single private constructor of ExpressionVirtualColumn that takes a nullable 'expression' and an optional pre-parsed expression.
   2) Make a `@JsonCreator` static factory method that Jackson will use.
   3) Make another static factory method that accepts a pre-parsed expression, and stores null for the 'expression'.
   4) Make `getExpression()` throw an exception if 'expression' is null. This means anyone trying to serialize it will realize what is going on before getting hit by latent bugs.
   5) Make sure to note in the javadoc of JoinFilterAnalysis that the virtual columns are not serializable.
   
   Open to other ideas.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374314381
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @param filter           Filter to be rewritten
+   * @param prefixes         Map of join table prefixes to clauses
+   * @param equiconditions   Map of equiconditions
+   * @param correlationCache Cache of column correlation analyses
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      Filter filter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    assert (filter instanceof SelectorFilter);
+    SelectorFilter selectorFilter = (SelectorFilter) filter;
 
 Review comment:
   Can `rewriteSelectorFilter`'s `filter` parameter be of type `SelectorFilter` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375026542
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
 
 Review comment:
   What case is this meant to handle? I don't understand why we'd need to switch `findMappingFor` from a rhs column to a lhs column.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374310135
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @param filter           Filter to be rewritten
+   * @param prefixes         Map of join table prefixes to clauses
+   * @param equiconditions   Map of equiconditions
+   * @param correlationCache Cache of column correlation analyses
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      Filter filter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    assert (filter instanceof SelectorFilter);
+    SelectorFilter selectorFilter = (SelectorFilter) filter;
+
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            filter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filter,
+        filter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
+          if (adapter.isBaseColumn(identifier)) {
+            terminate = true;
+            correlatedBaseColumns.add(findMappingFor);
+          }
+        }
+      }
+
+      if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) {
+        return null;
+      }
+
+      correlations.add(
+          new JoinFilterColumnCorrelationAnalysis(
+              rhsColumn,
+              correlatedBaseColumns,
+              correlatedBaseExpressions
+          )
+      );
+    }
+
+    return correlations;
+  }
+
+  private static boolean filterMatchesNull(Filter filter)
+  {
+    ValueMatcher valueMatcher = filter.makeMatcher(new AllNullColumnSelectorFactory());
 
 Review comment:
   Can `AllNullColumnSelectorFactory` be made a private class variable so it does not need to be allocated each time?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374312758
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
 
 Review comment:
   Can remove since this method never returns `null`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374280504
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java
 ##########
 @@ -306,4 +307,26 @@ private boolean doesMatch(String input)
     }
     return (lowerComparing >= 0) && (upperComparing >= 0);
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Similar comment about testing with `EqualsVerifier`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375007877
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
 
 Review comment:
   Skimming a bit, it seems like this would work if it was given a `List<JoinableClause>` instead of a `HashJoinSegmentStorageAdapter`. If so, that'd be a good change (it's better for methods like this to take conceptually smaller objects — unit testing is easier, and it makes it clearer what the 'real' dependencies are of the computation it is doing).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666784
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause from a filter that is in CNF
+   * @param prefixes         Map of table prefixes
+   * @param equiconditions   Equicondition map
+   * @param correlationCache Cache of column correlation analyses
 
 Review comment:
   Added a note about the cache being modified

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374280859
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java
 ##########
 @@ -218,4 +219,23 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask)
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Similar comment about testing with `EqualsVerifier`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667223
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
 
 Review comment:
   Made these use `pushDown` consistently

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374312112
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
 ##########
 @@ -52,6 +54,11 @@
   private final StorageAdapter baseAdapter;
   private final List<JoinableClause> clauses;
 
+  // A reference to the last JoinFilterSplit created during a makeCursors call,
+  // saved and exposed so that tests can verify the filter splitting behavior.
+  @VisibleForTesting
+  private JoinFilterAnalyzer.JoinFilterSplit previousJoinFilterSplitForTesting;
 
 Review comment:
   Would you still need this if `JoinFilterAnalyzerTest` was created and had unit tests for `JoinFilterAnalyzer`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375025326
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
 
 Review comment:
   equiConditions would match the spelling from JoinConditionAnalysis.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on issue #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on issue #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#issuecomment-583153589
 
 
   @gianm I addressed the latest round of comments, the TC failures were legit (the methods were no longer used after adjustments for https://github.com/apache/druid/pull/9301#discussion_r375014730)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376052078
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
 
 Review comment:
   I think it is specific to the filtering code, because it's handled consistently elsewhere. (The clauses check in order whether they match and the first one wins.) So this patch is introducing an inconsistency in behavior that didn't used to exist. But I do think it's fine to do in a follow on. Please raise a github issue and add a comment pointing to it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374998023
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
 ##########
 @@ -52,6 +54,11 @@
   private final StorageAdapter baseAdapter;
   private final List<JoinableClause> clauses;
 
+  // A reference to the last JoinFilterSplit created during a makeCursors call,
+  // saved and exposed so that tests can verify the filter splitting behavior.
+  @VisibleForTesting
+  private JoinFilterAnalyzer.JoinFilterSplit previousJoinFilterSplitForTesting;
 
 Review comment:
   I restructured the tests in `JoinFilterAnalyzerTest` to call `JoinFilterAnalyzer.splitFilter` directly, and got rid of this testing hook. 
   
   This means the filter split would be calculated twice in the tests (since I still call verifyCursors later), but I think that's fine for tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996725
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
 
 Review comment:
   Changed to Collections.singletonList()

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376130138
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
 ##########
 @@ -62,6 +64,23 @@ public ExpressionVirtualColumn(
     this.parsedExpression = Suppliers.memoize(() -> Parser.parse(expression, macroTable));
   }
 
+  /**
+   * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression.
+   */
+  public ExpressionVirtualColumn(
+      String name,
+      Expr parsedExpression,
+      ValueType outputType
+  )
+  {
+    this.name = Preconditions.checkNotNull(name, "name");
+    // Unfortunately this string representation can't be reparsed into the same expression, might be useful
+    // if the expression system supported that
+    this.expression = parsedExpression.toString();
 
 Review comment:
   I opened https://github.com/apache/druid/issues/9326 to track the expressions enhancement and mentioned it in a comment here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996856
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
 
 Review comment:
   Added an EqualsVerifier test in the new `JoinFilterAnalyzerTest` class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129261
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
 
 Review comment:
   Thanks, those are much better names, changed to the suggested description and names

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375669123
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
 
 Review comment:
   Still looking into this comment

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667795
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
 
 Review comment:
   I'm still looking into this comment

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376129847
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
+            if (!adapter.isBaseColumn(requiredBinding)) {
+              return null;
+            }
+          }
+
+          terminate = true;
+          correlatedBaseExpressions.add(lhs);
+        } else {
+          // simple identifier, see if we can correlate it with a column on the base table
+          findMappingFor = identifier;
 
 Review comment:
   I added more description and examples to the javadoc for findCorrelatedBaseTableColumns, hopefully it's more clear now

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374280677
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/InFilter.java
 ##########
 @@ -235,4 +236,26 @@ public DruidDoublePredicate makeDoublePredicate()
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Similar comment about testing with `EqualsVerifier`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374997493
 
 

 ##########
 File path: processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
 ##########
 @@ -1288,103 +1313,1163 @@ public void test_makeCursors_errorOnNonKeyBasedJoin()
     );
   }
 
-  private JoinableClause factToCountryNameUsingIsoCodeLookup(final JoinType joinType)
+  // Filter push down tests
 
 Review comment:
   Thanks!
   
   I split out the segment setup and some helper methods out of `HashJoinSegmentStorageAdapterTest` into `BaseHashJoinSegmentStorageAdapterTest` since the JoinFilterAnalyzer needs a HashJoinSegmentStorageAdapter, both `JoinFilterAnalyzerTest` and `HashJoinSegmentStorageAdapterTest` now extend the base.
   
   The filter push down tests have been moved into `JoinFilterAnalyzerTest`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375014730
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
 
 Review comment:
   Instead of doing an `instanceof`, could you add a method to Joinable that enables this use case? The interface is still really new and we should be evolving it to meet our needs.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r376041970
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
 
 Review comment:
   I think doing it in a follow-on is OK. Please just add a comment about the known limitation and create a followup github issue (the comment could just link to the issue).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374996498
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java
 ##########
 @@ -234,4 +235,23 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask)
       }
     };
   }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Added an EqualsVerifier test

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375666701
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
 
 Review comment:
   Added a comment about why NULL matching conditions are not handled now

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375669811
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
 ##########
 @@ -62,6 +64,23 @@ public ExpressionVirtualColumn(
     this.parsedExpression = Suppliers.memoize(() -> Parser.parse(expression, macroTable));
   }
 
+  /**
+   * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression.
+   */
+  public ExpressionVirtualColumn(
+      String name,
+      Expr parsedExpression,
+      ValueType outputType
+  )
+  {
+    this.name = Preconditions.checkNotNull(name, "name");
+    // Unfortunately this string representation can't be reparsed into the same expression, might be useful
+    // if the expression system supported that
+    this.expression = parsedExpression.toString();
 
 Review comment:
   I decided to just disable lhs expression push down for now (JoinFilterColumnCorrelationAnalysis.supportsPushDown returns false now if there are lhs expressions), but kept the rest of the code there and added an `@Ignore` annotation to tests that have lhs expressions (since the "only" thing blocking it is the lack of support from the expression system)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375017083
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
 
 Review comment:
   This should be `@Nullable` given the annotation in the constructor.
   
   Alternatively, consider `Optional<Filter>` rather than `@Nullable Filter`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667826
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
+    for (Equality eq : jca.getEquiConditions()) {
+      rhsColumns.add(tablePrefix + eq.getRightColumn());
+    }
+
+    List<JoinFilterColumnCorrelationAnalysis> correlations = new ArrayList<>();
+
+    for (String rhsColumn : rhsColumns) {
+      List<String> correlatedBaseColumns = new ArrayList<>();
+      List<Expr> correlatedBaseExpressions = new ArrayList<>();
+      boolean terminate = false;
+
+      String findMappingFor = rhsColumn;
+      while (!terminate) {
+        Expr lhs = equiconditions.get(findMappingFor);
+        if (lhs == null) {
+          break;
+        }
+        String identifier = lhs.getBindingIfIdentifier();
+        if (identifier == null) {
+          // We push down if the function only requires base table columns
+          Expr.BindingDetails bindingDetails = lhs.analyzeInputs();
+          Set<String> requiredBindings = bindingDetails.getRequiredBindings();
+          for (String requiredBinding : requiredBindings) {
 
 Review comment:
   Changed to the suggested stream usage

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667161
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
 
 Review comment:
   Adjusted this to return `Optional<Filter>` for the filter getters

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375027922
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
 
 Review comment:
   When would there be more than one base column?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374997723
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          baseAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  @Nullable
 
 Review comment:
   Removed the unnecessary `@Nullable`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375012488
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
 
 Review comment:
   Would be nice to say why (I presume it's something like: conditions that match NULL are tricky to push down when doing OUTER JOINs, and we'd rather not worry about that right now).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375667758
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
+        List<JoinFilterColumnCorrelationAnalysis> correlations = correlationCache.computeIfAbsent(
+            prefixAndClause.getKey(),
+            p -> findCorrelatedBaseTableColumns(
+                baseAdapter,
+                p,
+                prefixes.get(p),
+                equiconditions
+            )
+        );
+
+        if (correlations == null) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        List<Filter> newFilters = new ArrayList<>();
+        List<VirtualColumn> pushdownVirtualColumns = new ArrayList<>();
+
+        for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations) {
+          if (correlationAnalysis.supportsPushDown()) {
+            List<String> correlatedValues = getCorrelatedValuesForPushDown(
+                selectorFilter.getDimension(),
+                selectorFilter.getValue(),
+                correlationAnalysis.getJoinColumn(),
+                prefixAndClause.getValue()
+            );
+
+            if (correlatedValues == null) {
+              return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+            }
+
+            for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) {
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  correlatedBaseColumn,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+
+            for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) {
+              // need to create a virtual column for the expressions when pushing down
+              String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size());
+
+              VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn(
+                  vcName,
+                  correlatedBaseExpr,
+                  ValueType.STRING
+              );
+              pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn);
+
+              InFilter rewrittenFilter = (InFilter) new InDimFilter(
+                  vcName,
+                  correlatedValues,
+                  null,
+                  null
+              ).toFilter();
+              newFilters.add(rewrittenFilter);
+            }
+          }
+        }
+
+        if (newFilters.isEmpty()) {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter);
+        }
+
+        return new JoinFilterAnalysis(
+            true,
+            true,
+            selectorFilter,
+            newFilters.size() == 1 ? newFilters.get(0) : new AndFilter(newFilters),
+            pushdownVirtualColumns
+        );
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        selectorFilter,
+        selectorFilter,
+        null
+    );
+  }
+
+  private static String getCorrelatedBaseExprVirtualColumnName(int counter)
+  {
+    // May want to have this check other column names to absolutely prevent name conflicts
+    return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter;
+  }
+
+  /**
+   * Helper method for rewriting filters on join table columns into filters on base table columns.
+   *
+   * @param filterColumn           A join table column that we're filtering on
+   * @param filterValue            The value to filter on
+   * @param correlatedJoinColumn   A join table column that appears as the RHS of an equicondition, which we can correlate
+   *                               with a column on the base table
+   * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on
+   *
+   * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue
+   * Returns null if we cannot determine the correlated values.
+   */
+  @Nullable
+  private static List<String> getCorrelatedValuesForPushDown(
+      String filterColumn,
+      String filterValue,
+      String correlatedJoinColumn,
+      JoinableClause clauseForFilteredTable
+  )
+  {
+    String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length());
+    String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length());
+
+    // would be good to allow non-key column indices on the Joinables for better perf
+    if (clauseForFilteredTable.getJoinable() instanceof LookupJoinable) {
+      LookupJoinable lookupJoinable = (LookupJoinable) clauseForFilteredTable.getJoinable();
+      List<String> correlatedValues;
+      if (LookupColumnSelectorFactory.KEY_COLUMN.equals(filterColumnNoPrefix)) {
+        if (LookupColumnSelectorFactory.KEY_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = ImmutableList.of(lookupJoinable.getExtractor().apply(filterColumnNoPrefix));
+        }
+      } else {
+        if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(correlatedColumnNoPrefix)) {
+          correlatedValues = ImmutableList.of(filterValue);
+        } else {
+          correlatedValues = lookupJoinable.getExtractor().unapply(filterValue);
+        }
+      }
+      return correlatedValues;
+    }
+
+    if (clauseForFilteredTable.getJoinable() instanceof IndexedTableJoinable) {
+      IndexedTableJoinable indexedTableJoinable = (IndexedTableJoinable) clauseForFilteredTable.getJoinable();
+      IndexedTable indexedTable = indexedTableJoinable.getTable();
+
+      int filterColumnPosition = indexedTable.allColumns().indexOf(filterColumnNoPrefix);
+      int correlatedColumnPosition = indexedTable.allColumns().indexOf(correlatedColumnNoPrefix);
+
+      if (filterColumnPosition < 0 || correlatedColumnPosition < 0) {
+        return null;
+      }
+
+      if (indexedTable.keyColumns().contains(filterColumnNoPrefix)) {
+        IndexedTable.Index index = indexedTable.columnIndex(filterColumnPosition);
+        IndexedTable.Reader reader = indexedTable.columnReader(correlatedColumnPosition);
+        IntList rowIndex = index.find(filterValue);
+        List<String> correlatedValues = new ArrayList<>();
+        for (int i = 0; i < rowIndex.size(); i++) {
+          int rowNum = rowIndex.getInt(i);
+          correlatedValues.add(reader.read(rowNum).toString());
+        }
+        return correlatedValues;
+      } else {
+        IndexedTable.Reader dimNameReader = indexedTable.columnReader(filterColumnPosition);
+        IndexedTable.Reader correlatedColumnReader = indexedTable.columnReader(correlatedColumnPosition);
+        Set<String> correlatedValueSet = new HashSet<>();
+        for (int i = 0; i < indexedTable.numRows(); i++) {
+          if (filterValue.equals(dimNameReader.read(i).toString())) {
+            correlatedValueSet.add(correlatedColumnReader.read(i).toString());
+          }
+        }
+
+        return new ArrayList<>(correlatedValueSet);
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * For all RHS columns that appear in the join's equiconditions, correlate them with base table columns if possible.
+   *
+   * @param adapter              The adapter for the join. Used to determine if a column is a base table column.
+   * @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
+   */
+  @Nullable
+  private static List<JoinFilterColumnCorrelationAnalysis> findCorrelatedBaseTableColumns(
+      HashJoinSegmentStorageAdapter adapter,
+      String tablePrefix,
+      JoinableClause clauseForTablePrefix,
+      Map<String, Expr> equiconditions
+  )
+  {
+    JoinConditionAnalysis jca = clauseForTablePrefix.getCondition();
+
+    List<String> rhsColumns = new ArrayList<>();
 
 Review comment:
   Ah, changed this to Set

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374305806
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,739 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+
+  public static JoinFilterSplit splitFilter(
+      Filter originalFilter,
+      HashJoinSegmentStorageAdapter baseAdapter,
+      List<JoinableClause> clauses
+  )
+  {
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : clauses) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = new ArrayList<>();
+      normalizedOrClauses.add(normalizedFilter);
 
 Review comment:
   Maybe use `Collections.singletonList()` instead

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r374281048
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java
 ##########
 @@ -127,4 +128,35 @@ public String toString()
   {
     return StringUtils.format("%s = %s", dimension, value);
   }
+
+  public String getDimension()
+  {
+    return dimension;
+  }
+
+  public String getValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
 
 Review comment:
   Similar comment about testing with `EqualsVerifier`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375689903
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
 
 Review comment:
   I tried a test where I had the following join condition:
   
   ```
   "r1.regionIsoCode" == regionIsoCode && "r1.regionIsoCode" == countryIsoCode
   ```
   
   In one of the Equality objects in the equiConditions, the right hand column ends up as `regionIsoCode_0` with an extra `_0` suffix, and the query fails with: 
   
   ```
   "Cannot build hash-join matcher on non-key-based condition: Equality{leftExpr=countryIsoCode, rightColumn='regionIsoCode_0'}"
   ```
   I added a new test which shows this (`JoinFilterAnalyzerTest.test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnRHS`)
   
   Can we address this in a follow-on PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9301: Join filter pushdown initial implementation

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9301: Join filter pushdown initial implementation
URL: https://github.com/apache/druid/pull/9301#discussion_r375022267
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/join/JoinFilterAnalyzer.java
 ##########
 @@ -0,0 +1,743 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import org.apache.druid.math.expr.Expr;
+import org.apache.druid.query.dimension.DimensionSpec;
+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.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+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.InFilter;
+import org.apache.druid.segment.filter.OrFilter;
+import org.apache.druid.segment.filter.SelectorFilter;
+import org.apache.druid.segment.join.lookup.LookupColumnSelectorFactory;
+import org.apache.druid.segment.join.lookup.LookupJoinable;
+import org.apache.druid.segment.join.table.IndexedTable;
+import org.apache.druid.segment.join.table.IndexedTableJoinable;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class JoinFilterAnalyzer
+{
+  private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-";
+  private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory();
+
+  public static JoinFilterSplit splitFilter(
+      HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter,
+      @Nullable Filter originalFilter
+  )
+  {
+    if (originalFilter == null) {
+      return new JoinFilterAnalyzer.JoinFilterSplit(
+          null,
+          null,
+          ImmutableList.of()
+      );
+    }
+
+    Filter normalizedFilter = Filters.convertToCNF(originalFilter);
+
+    // build the prefix and equicondition maps
+    Map<String, Expr> equiconditions = new HashMap<>();
+    Map<String, JoinableClause> prefixes = new HashMap<>();
+    for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) {
+      prefixes.put(clause.getPrefix(), clause);
+      for (Equality equality : clause.getCondition().getEquiConditions()) {
+        equiconditions.put(clause.getPrefix() + equality.getRightColumn(), equality.getLeftExpr());
+      }
+    }
+
+    // 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<Filter> normalizedOrClauses;
+    if (normalizedFilter instanceof AndFilter) {
+      normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters();
+    } else {
+      normalizedOrClauses = Collections.singletonList(normalizedFilter);
+    }
+
+    // Pushdown filters, rewriting if necessary
+    List<Filter> leftFilters = new ArrayList<>();
+    List<Filter> rightFilters = new ArrayList<>();
+    List<VirtualColumn> pushDownVirtualColumns = new ArrayList<>();
+    Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache = new HashMap<>();
+
+    for (Filter orClause : normalizedOrClauses) {
+      JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause(
+          hashJoinSegmentStorageAdapter,
+          orClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+      if (joinFilterAnalysis.isCanPushDown()) {
+        leftFilters.add(joinFilterAnalysis.getPushdownFilter());
+        if (joinFilterAnalysis.getPushdownVirtualColumns() != null) {
+          pushDownVirtualColumns.addAll(joinFilterAnalysis.getPushdownVirtualColumns());
+        }
+      }
+      if (joinFilterAnalysis.isRetainAfterJoin()) {
+        rightFilters.add(joinFilterAnalysis.getOriginalFilter());
+      }
+    }
+
+    return new JoinFilterSplit(
+        leftFilters.isEmpty() ? null : leftFilters.size() == 1 ? leftFilters.get(0) : new AndFilter(leftFilters),
+        rightFilters.isEmpty() ? null : rightFilters.size() == 1 ? rightFilters.get(0) : new AndFilter(rightFilters),
+        pushDownVirtualColumns
+    );
+  }
+
+  /**
+   * Holds the result of splitting a filter into:
+   * - a portion that can be pushed down to the base table
+   * - a portion that will be applied post-join
+   * - additional virtual columns that need to be created on the base table to support the pushed down filters.
+   */
+  public static class JoinFilterSplit
+  {
+    final Filter baseTableFilter;
+    final Filter joinTableFilter;
+    final List<VirtualColumn> pushDownVirtualColumns;
+
+    public JoinFilterSplit(
+        Filter baseTableFilter,
+        @Nullable Filter joinTableFilter,
+        List<VirtualColumn> pushDownVirtualColumns
+    )
+    {
+      this.baseTableFilter = baseTableFilter;
+      this.joinTableFilter = joinTableFilter;
+      this.pushDownVirtualColumns = pushDownVirtualColumns;
+    }
+
+    public Filter getBaseTableFilter()
+    {
+      return baseTableFilter;
+    }
+
+    public Filter getJoinTableFilter()
+    {
+      return joinTableFilter;
+    }
+
+    public List<VirtualColumn> getPushDownVirtualColumns()
+    {
+      return pushDownVirtualColumns;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "JoinFilterSplit{" +
+             "baseTableFilter=" + baseTableFilter +
+             ", joinTableFilter=" + joinTableFilter +
+             ", pushDownVirtualColumns=" + pushDownVirtualColumns +
+             '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JoinFilterSplit that = (JoinFilterSplit) o;
+      return Objects.equals(getBaseTableFilter(), that.getBaseTableFilter()) &&
+             Objects.equals(getJoinTableFilter(), that.getJoinTableFilter()) &&
+             Objects.equals(getPushDownVirtualColumns(), that.getPushDownVirtualColumns());
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(getBaseTableFilter(), getJoinTableFilter(), getPushDownVirtualColumns());
+    }
+  }
+
+  /**
+   * Analyze a single filter clause from a filter that is in conjunctive normal form (AND of ORs),
+   * returning a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
+   *
+   * @param adapter          Adapter for the join
+   * @param filterClause     Individual filter clause 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(
+      HashJoinSegmentStorageAdapter adapter,
+      Filter filterClause,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    // NULL matching conditions are not currently pushed down
+    if (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(
+          adapter,
+          (SelectorFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    if (filterClause instanceof OrFilter) {
+      return rewriteOrFilter(
+          adapter,
+          (OrFilter) filterClause,
+          prefixes,
+          equiconditions,
+          correlationCache
+      );
+    }
+
+    for (String requiredColumn : filterClause.getRequiredColumns()) {
+      if (!adapter.isBaseColumn(requiredColumn)) {
+        return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause);
+      }
+    }
+    return new JoinFilterAnalysis(
+        true,
+        false,
+        filterClause,
+        filterClause,
+        null
+    );
+  }
+
+  /**
+   * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
+   * the base table.
+   *
+   * @param adapter          Adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteOrFilter(
+      HashJoinSegmentStorageAdapter adapter,
+      OrFilter orFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    boolean retainRhs = false;
+
+    List<Filter> newFilters = new ArrayList<>();
+    for (Filter filter : orFilter.getFilters()) {
+      boolean allBaseColumns = true;
+      for (String requiredColumn : filter.getRequiredColumns()) {
+        if (!adapter.isBaseColumn(requiredColumn)) {
+          allBaseColumns = false;
+        }
+      }
+
+      if (!allBaseColumns) {
+        retainRhs = true;
+        if (filter instanceof SelectorFilter) {
+          JoinFilterAnalysis rewritten = rewriteSelectorFilter(
+              adapter,
+              (SelectorFilter) filter,
+              prefixes,
+              equiconditions,
+              correlationCache
+          );
+          if (!rewritten.isCanPushDown()) {
+            return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+          } else {
+            newFilters.add(rewritten.getPushdownFilter());
+          }
+        } else {
+          return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter);
+        }
+      } else {
+        newFilters.add(filter);
+      }
+    }
+
+    return new JoinFilterAnalysis(
+        true,
+        retainRhs,
+        orFilter,
+        new OrFilter(newFilters),
+        null
+    );
+  }
+
+  /**
+   * Rewrites a selector filter on a join table into an IN filter on the base table.
+   *
+   * @param baseAdapter      The adapter for the join
+   * @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
+   *
+   * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
+   */
+  private static JoinFilterAnalysis rewriteSelectorFilter(
+      HashJoinSegmentStorageAdapter baseAdapter,
+      SelectorFilter selectorFilter,
+      Map<String, JoinableClause> prefixes,
+      Map<String, Expr> equiconditions,
+      Map<String, List<JoinFilterColumnCorrelationAnalysis>> correlationCache
+  )
+  {
+    String filteringColumn = selectorFilter.getDimension();
+    for (Map.Entry<String, JoinableClause> prefixAndClause : prefixes.entrySet()) {
+      if (filteringColumn.startsWith(prefixAndClause.getKey())) {
 
 Review comment:
   Should use `prefixAndClause.getValue().includesColumn(filteringColumn)` — it's more semantically intentful and also the logic is slightly different.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org