[ 
https://issues.apache.org/jira/browse/DRILL-6381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16647444#comment-16647444
 ] 

ASF GitHub Bot commented on DRILL-6381:
---------------------------------------

vdiravka commented on a change in pull request #1466: DRILL-6381: Add support 
for index based planning and execution
URL: https://github.com/apache/drill/pull/1466#discussion_r223670914
 
 

 ##########
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##########
 @@ -0,0 +1,872 @@
+/*
+ * 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.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+    NONE,
+    PARTIAL,
+    FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+    Map<RexNode, LogicalExpression> mapRexExpr = 
exprMarker.getIndexableExpression();
+    List<LogicalExpression> infoCols = Lists.newArrayList();
+    infoCols.addAll(mapRexExpr.values());
+    if (indexDesc.allColumnsIndexed(infoCols)) {
+      return ConditionIndexed.FULL;
+    } else if (indexDesc.someColumnsIndexed(infoCols)) {
+      return ConditionIndexed.PARTIAL;
+    } else {
+      return ConditionIndexed.NONE;
+    }
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *    this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+    GroupScan groupScan = scanRel.getGroupScan();
+    if (groupScan instanceof DbGroupScan) {
+      DbGroupScan dbscan = ((DbGroupScan) groupScan);
+      //if we already applied index convert rule, and this scan is indexScan 
or restricted scan already,
+      //no more trying index convert rule
+      return dbscan.supportsSecondaryIndex() && (!dbscan.isIndexScan()) && 
(!dbscan.isRestrictedScan());
+    }
+    return false;
+  }
+
+  /**
+   * For a particular table scan for table T1 and an index on that table, find 
out if it is a covering index
+   * @return
+   */
+  static public boolean isCoveringIndex(IndexCallContext indexContext, 
FunctionalIndexInfo functionInfo) {
+    if(functionInfo.hasFunctional()) {
+      //need info from full query
+      return queryCoveredByIndex(indexContext, functionInfo);
+    }
+    DbGroupScan groupScan = (DbGroupScan) getGroupScan(indexContext.getScan());
+    List<LogicalExpression> tableCols = Lists.newArrayList();
+    tableCols.addAll(groupScan.getColumns());
+    return functionInfo.getIndexDesc().isCoveringIndex(tableCols);
+  }
+
+
+  /**
+   * This method is called only when the index has at least one functional 
indexed field. If there is no function field,
+   * we don't need to worry whether there could be paths not found in Scan.
+   * In functional case, we have to check all available (if needed) operators 
to find out if the query is covered or not.
+   * E.g. cast(a.b as INT) in project, a.b in Scan's rowType or columns, and 
cast(a.b as INT)
+   * is an indexed field named '$0'. In this case, by looking at Scan, we see 
only 'a.b' which is not in index. We have to
+   * look into Project, and if we see 'a.b' is only used in functional index 
expression cast(a.b as INT), then we know
+   * this Project+Scan is covered.
+   * @param indexContext
+   * @param functionInfo
+   * @return false if the query could not be covered by the index (should not 
create covering index plan)
+   */
+  static private boolean queryCoveredByIndex(IndexCallContext indexContext,
+                              FunctionalIndexInfo functionInfo) {
+    //for indexed functions, if relevant schemapaths are included in index(in 
indexed fields or non-indexed fields),
+    // check covering based on the local information we have:
+    //   if references to schema paths in functional indexes disappear beyond 
capProject
+
+    if (indexContext.getFilter() != null && indexContext.getUpperProject() == 
null) {
+      if( !isFullQuery(indexContext)) {
+        return false;
+      }
+    }
+
+    DrillParseContext parserContext =
+        new 
DrillParseContext(PrelUtil.getPlannerSettings(indexContext.getCall().rel(0).getCluster()));
+
+    Set<LogicalExpression> exprs = Sets.newHashSet();
+    if (indexContext.getUpperProject() != null) {
+      if (indexContext.getLowerProject() == null) {
+        for (RexNode rex : indexContext.getUpperProject().getProjects()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+        //now collect paths in filter since upperProject may drop some paths 
in filter
+        IndexableExprMarker filterMarker = new 
IndexableExprMarker(indexContext.getScan());
+        indexContext.getFilterCondition().accept(filterMarker);
+        for (RexNode rex : filterMarker.getIndexableExpression().keySet()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+      } else {
+        //we have underneath project, so we have to do more to convert 
expressions
+        for (RexNode rex : indexContext.getUpperProject().getProjects()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
indexContext.getLowerProject(), indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+
+        // Now collect paths in filter since upperProject may drop some paths 
in filter.
+        // Since this is (upper)Proj+Filter+(lower)Proj+Scan case, and 
IndexableExprMarker works
+        // only with expressions that referencing directly to Scan, it has to 
use indexContext.origPushedCondition
+        IndexableExprMarker filterMarker = new 
IndexableExprMarker(indexContext.getScan());
+        indexContext.getOrigCondition().accept(filterMarker);
+
+        for (RexNode rex : filterMarker.getIndexableExpression().keySet()) {
+          // Since rex represents the filter expression directly referencing 
the scan row type,
+          // (the condition has been pushed down of lowerProject), set the 
lowerProject as null.
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+      }
+    }
+    else if (indexContext.getLowerProject() != null) {
+      for (RexNode rex : indexContext.getLowerProject().getProjects()) {
+        LogicalExpression expr = DrillOptiq.toDrill(parserContext, 
indexContext.getScan(), rex);
+        exprs.add(expr);
+      }
+    }
+    else {//upperProject and lowerProject both are null, the only place to 
find columns being used in query is scan
+      exprs.addAll(indexContext.getScanColumns());
+    }
+
+    Map<LogicalExpression, Set<SchemaPath>> exprPathMap = 
functionInfo.getPathsInFunctionExpr();
+    PathInExpr exprSearch = new PathInExpr(exprPathMap);
+
+    for(LogicalExpression expr: exprs) {
+      if(expr.accept(exprSearch, null) == false) {
+        return false;
+      }
+    }
+    //if we come to here, paths in indexed function expressions are covered in 
capProject.
+    //now we check other paths.
+
+    //check the leftout paths (appear in capProject other than functional 
index expression) are covered by other index fields or not
+    List<LogicalExpression> leftPaths = 
Lists.newArrayList(exprSearch.getRemainderPaths());
+
+    
indexContext.setLeftOutPathsInFunctions(exprSearch.getRemainderPathsInFunctions());
+    return functionInfo.getIndexDesc().isCoveringIndex(leftPaths);
+  }
+
+  static private boolean isFullQuery(IndexCallContext indexContext) {
+    RelNode rootInCall = indexContext.getCall().rel(0);
+    //check if the tip of the operator stack we have is also the top of the 
whole query, if yes, return true
+    if (indexContext.getCall().getPlanner().getRoot() instanceof RelSubset) {
+      final RelSubset rootSet = (RelSubset) 
indexContext.getCall().getPlanner().getRoot();
+      if (rootSet.getRelList().contains(rootInCall)) {
+        return true;
+      }
+    } else {
+      if (indexContext.getCall().getPlanner().getRoot().equals(rootInCall)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Build collation property for the 'lower' project, the one closer to the 
Scan
+   * @param projectRexs
+   * @param input
+   * @param indexInfo
+   * @return the output RelCollation
+   */
+  public static RelCollation buildCollationLowerProject(List<RexNode> 
projectRexs, RelNode input, FunctionalIndexInfo indexInfo) {
+    //if leading fields of index are here, add them to RelCollation
+    List<RelFieldCollation> newFields = Lists.newArrayList();
+    if (!indexInfo.hasFunctional()) {
+      Map<LogicalExpression, Integer> projectExprs = Maps.newLinkedHashMap();
+      DrillParseContext parserContext = new 
DrillParseContext(PrelUtil.getPlannerSettings(input.getCluster()));
+      int idx=0;
+      for(RexNode rex : projectRexs) {
+        projectExprs.put(DrillOptiq.toDrill(parserContext, input, rex), idx);
+        idx++;
+      }
+      int idxFieldCount = 0;
+      for (LogicalExpression expr : 
indexInfo.getIndexDesc().getIndexColumns()) {
+        if (!projectExprs.containsKey(expr)) {
+          break;
+        }
+        RelFieldCollation.Direction dir = 
indexInfo.getIndexDesc().getCollation().getFieldCollations().get(idxFieldCount).direction;
+        if ( dir == null) {
+          break;
+        }
+        newFields.add(new RelFieldCollation(projectExprs.get(expr), dir,
+            RelFieldCollation.NullDirection.UNSPECIFIED));
+      }
+      idxFieldCount++;
+    } else {
+      // TODO: handle functional index
+    }
+
+    return RelCollations.of(newFields);
+  }
+
+  /**
+   * Build collation property for the 'upper' project, the one above the filter
+   * @param projectRexs
+   * @param inputCollation
+   * @param indexInfo
+   * @param collationFilterMap
+   * @return the output RelCollation
+   */
+  public static RelCollation buildCollationUpperProject(List<RexNode> 
projectRexs,
+                                                        RelCollation 
inputCollation, FunctionalIndexInfo indexInfo,
+                                                        Map<Integer, 
List<RexNode>> collationFilterMap) {
+    List<RelFieldCollation> outputFieldCollations = Lists.newArrayList();
+
+    if (inputCollation != null) {
+      List<RelFieldCollation> inputFieldCollations = 
inputCollation.getFieldCollations();
+      if (!indexInfo.hasFunctional()) {
+        for (int projectExprIdx = 0; projectExprIdx < projectRexs.size(); 
projectExprIdx++) {
+          RexNode n = projectRexs.get(projectExprIdx);
+          if (n instanceof RexInputRef) {
+            RexInputRef ref = (RexInputRef)n;
+            boolean eligibleForCollation = true;
+            int maxIndex = getIndexFromCollation(ref.getIndex(), 
inputFieldCollations);
+            if (maxIndex < 0) {
+              eligibleForCollation = false;
+              continue;
+            }
+            // check if the prefix has equality conditions
+            for (int i = 0; i < maxIndex; i++) {
+              int fieldIdx = inputFieldCollations.get(i).getFieldIndex();
+              List<RexNode> conditions = collationFilterMap != null ? 
collationFilterMap.get(fieldIdx) : null;
+              if ((conditions == null || conditions.size() == 0) &&
+                  i < maxIndex-1) {
+                // if an intermediate column has no filter condition, it would 
select all values
+                // of that column, so a subsequent column cannot be eligible 
for collation
+                eligibleForCollation = false;
+                break;
+              } else {
+                for (RexNode r : conditions) {
+                  if (!(r.getKind() == SqlKind.EQUALS)) {
+                    eligibleForCollation = false;
+                    break;
+                  }
+                }
+              }
+            }
+            // for every projected expr, if it is eligible for collation, get 
the
+            // corresponding field collation from the input
+            if (eligibleForCollation) {
+              for (RelFieldCollation c : inputFieldCollations) {
+                if (ref.getIndex() == c.getFieldIndex()) {
+                  RelFieldCollation outFieldCollation = new 
RelFieldCollation(projectExprIdx, c.getDirection(), c.nullDirection);
+                  outputFieldCollations.add(outFieldCollation);
+                }
+              }
+            }
+          }
+        }
+      } else {
+        // TODO: handle functional index
+      }
+    }
+    return RelCollations.of(outputFieldCollations);
+  }
+
+  public static int getIndexFromCollation(int refIndex, 
List<RelFieldCollation> inputFieldCollations) {
+    for (int i=0; i < inputFieldCollations.size(); i++) {
+      if (refIndex == inputFieldCollations.get(i).getFieldIndex()) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<RexNode> getProjects(DrillProjectRelBase proj) {
+    return proj.getProjects();
+  }
+
+  public static boolean generateLimit(OrderedRel sort) {
+    RexNode fetchNode = sort.getFetch();
+    int fetchValue = (fetchNode == null) ? -1 : RexLiteral.intValue(fetchNode);
+    return fetchValue >=0;
+  }
+
+  public static RexNode getOffset(OrderedRel sort) {
+    return sort.getOffset();
+  }
+
+  public static RexNode getFetch(OrderedRel sort) {
+    return sort.getFetch();
+  }
+
+
+  /**
+   * generate logical expressions for sort rexNodes in SortRel, the result is 
store to IndexPlanCallContext
+   * @param indexContext
+   */
+  public static void updateSortExpression(IndexCallContext indexContext, 
List<RelFieldCollation> coll) {
+
+    if (coll == null) {
+      return;
+    }
+
+    DrillParseContext parserContext =
+        new 
DrillParseContext(PrelUtil.getPlannerSettings(indexContext.getCall().rel(0).getCluster()));
+
+    indexContext.createSortExprs();
+    for (RelFieldCollation collation : coll) {
+      int idx = collation.getFieldIndex();
+      DrillProjectRelBase oneProject;
+      if (indexContext.getUpperProject() != null && 
indexContext.getLowerProject() != null) {
+        LogicalExpression expr = RexToExpression.toDrill(parserContext, 
indexContext.getLowerProject(), indexContext.getScan(),
+            indexContext.getUpperProject().getProjects().get(idx));
+        indexContext.getSortExprs().add(expr);
+      }
+      else {//one project is null now
+        oneProject = (indexContext.getUpperProject() != null)? 
indexContext.getUpperProject() : indexContext.getLowerProject();
+        if(oneProject != null) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
null, indexContext.getScan(),
+              getProjects(oneProject).get(idx));
+          indexContext.getSortExprs().add(expr);
+        }
+        else {//two projects are null
+          SchemaPath path;
+          RelDataTypeField f = 
indexContext.getScan().getRowType().getFieldList().get(idx);
+          String pathSeg = f.getName().replaceAll("`", "");
+          final String[] segs = pathSeg.split("\\.");
+          path = SchemaPath.getCompoundPath(segs);
+          indexContext.getSortExprs().add(path);
+        }
+      }
+    }
+  }
+
+  /**
+   * generate logical expressions for sort rexNodes in SortRel, the result is 
store to IndexPlanCallContext
+   * @param indexContext
+   */
+  public static void updateSortExpression(IndexPhysicalPlanCallContext 
indexContext, List<RelFieldCollation> coll) {
+
+    if (coll == null) {
+      return;
+    }
+
+    DrillParseContext parserContext =
+            new 
DrillParseContext(PrelUtil.getPlannerSettings(indexContext.call.rel(0).getCluster()));
+
+    indexContext.sortExprs = Lists.newArrayList();
+    for (RelFieldCollation collation : coll) {
+      int idx = collation.getFieldIndex();
+      ProjectPrel oneProject;
+      if (indexContext.upperProject != null && indexContext.lowerProject != 
null) {
+        LogicalExpression expr = RexToExpression.toDrill(parserContext, 
indexContext.lowerProject, indexContext.scan,
+                indexContext.upperProject.getProjects().get(idx));
+        indexContext.sortExprs.add(expr);
+      }
+      else {//one project is null now
+        oneProject = (indexContext.upperProject != null)? 
indexContext.upperProject : indexContext.lowerProject;
+        if(oneProject != null) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, 
null, indexContext.scan,
+                  oneProject.getProjects().get(idx));
+          indexContext.sortExprs.add(expr);
+        }
+        else {//two projects are null
+          SchemaPath path;
+          RelDataTypeField f = 
indexContext.scan.getRowType().getFieldList().get(idx);
+          String pathSeg = f.getName().replaceAll("`", "");
+          final String[] segs = pathSeg.split("\\.");
+          path = SchemaPath.getCompoundPath(segs);
+          indexContext.sortExprs.add(path);
+        }
+      }
+    }
+  }
+
+  /**
+   *
+   * @param expr
+   * @param context
+   * @return if there is filter and expr is only in equality condition of the 
filter, return true
+   */
+  private static boolean exprOnlyInEquality(LogicalExpression expr, 
IndexCallContext context) {
+    //if there is no filter, expr wont be in equality
+    if(context.getFilter() == null) {
 
 Review comment:
   space

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


> Add capability to do index based planning and execution
> -------------------------------------------------------
>
>                 Key: DRILL-6381
>                 URL: https://issues.apache.org/jira/browse/DRILL-6381
>             Project: Apache Drill
>          Issue Type: New Feature
>          Components: Execution - Relational Operators, Query Planning &amp; 
> Optimization
>            Reporter: Aman Sinha
>            Assignee: Aman Sinha
>            Priority: Major
>             Fix For: 1.15.0
>
>
> If the underlying data source supports indexes (primary and secondary 
> indexes), Drill should leverage those during planning and execution in order 
> to improve query performance.  
> On the planning side, Drill planner should be enhanced to provide an 
> abstraction layer which express the index metadata and statistics.  Further, 
> a cost-based index selection is needed to decide which index(es) are 
> suitable.  
> On the execution side, appropriate operator enhancements would be needed to 
> handle different categories of indexes such as covering, non-covering 
> indexes, taking into consideration the index data may not be co-located with 
> the primary table, i.e a global index.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to