[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228358390
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
 ##
 @@ -57,7 +57,9 @@ public RelOptCost computeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
 
 final int  rowWidth = child.getRowType().getFieldCount() * 
DrillCostBase.AVG_FIELD_WIDTH;
 final double cpuCost = broadcastFactor * DrillCostBase.SVR_CPU_COST * 
inputRows;
-final double networkCost = broadcastFactor * 
DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * numEndPoints;
+
+//we assume localhost network cost is 1/10 of regular network cost
+final double networkCost = broadcastFactor * 
DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * (numEndPoints - 0.9);
 
 Review comment:
   I have added comments in the code regarding this. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228357573
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
 ##
 @@ -57,7 +57,9 @@ public RelOptCost computeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
 
 final int  rowWidth = child.getRowType().getFieldCount() * 
DrillCostBase.AVG_FIELD_WIDTH;
 final double cpuCost = broadcastFactor * DrillCostBase.SVR_CPU_COST * 
inputRows;
-final double networkCost = broadcastFactor * 
DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * numEndPoints;
+
+//we assume localhost network cost is 1/10 of regular network cost
+final double networkCost = broadcastFactor * 
DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * (numEndPoints - 0.9);
 
 Review comment:
   @gparai,  forgot to respond to this.  The cost formula is: 
   (cost of broadcasting num_bytes to N - 1 nodes)  +  (cost of local 
broadcast to all minor fragments on my own node)
  = (C * num_bytes * (N - 1) )  +  (C * num_bytes * 0.1)   where the 0.1 
factor comes from the assumption that local broadcast is 10% network cost of 
the remote broadcast. 
 = C * num_bytes * (N - 0.9)
   
   While the formula seems reasonable, it is biasing the cost in favor of 
Broadcast compared to HashPartition.  We should re-visit this and ideally a 
similar change should be done for HashPartition also.  I don't recall the exact 
use case which motivated the change.. it may have been the Index Intersection.  
I can create a JIRA to re-visit this. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228350448
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
 ##
 @@ -166,4 +169,25 @@ public void onMatch(RelOptRuleCall call) {
 return list;
   }
 
+  public static Project replace(Project topProject, Project bottomProject) {
 
 Review comment:
   Added javadoc. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228349995
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
 ##
 @@ -0,0 +1,145 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.util.Utilities;
+
+import java.util.List;
+
+public abstract class MapRDBPushProjectIntoScan extends 
StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(MapRDBPushProjectIntoScan.class);
+
+  private MapRDBPushProjectIntoScan(RelOptRuleOperand operand, String 
description) {
+super(operand, description);
+  }
+
+  public static final StoragePluginOptimizerRule PROJECT_ON_SCAN = new 
MapRDBPushProjectIntoScan(
+  RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class)), 
"MapRDBPushProjIntoScan:Proj_On_Scan") {
+@Override
+public void onMatch(RelOptRuleCall call) {
+  final ScanPrel scan = (ScanPrel) call.rel(1);
+  final ProjectPrel project = (ProjectPrel) call.rel(0);
+  if (!(scan.getGroupScan() instanceof MapRDBGroupScan)) {
+return;
+  }
+  doPushProjectIntoGroupScan(call, project, scan, (MapRDBGroupScan) 
scan.getGroupScan());
+  if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
+BinaryTableGroupScan groupScan = (BinaryTableGroupScan) 
scan.getGroupScan();
+
+  } else {
+assert (scan.getGroupScan() instanceof JsonTableGroupScan);
+JsonTableGroupScan groupScan = (JsonTableGroupScan) 
scan.getGroupScan();
+
+doPushProjectIntoGroupScan(call, project, scan, groupScan);
+  }
+}
+
+@Override
+public boolean matches(RelOptRuleCall call) {
+  final ScanPrel scan = (ScanPrel) call.rel(1);
+  if (scan.getGroupScan() instanceof BinaryTableGroupScan ||
+  scan.getGroupScan() instanceof JsonTableGroupScan) {
+return super.matches(call);
+  }
+  return false;
+}
+  };
+
+  protected void doPushProjectIntoGroupScan(RelOptRuleCall call,
+  ProjectPrel project, ScanPrel scan, MapRDBGroupScan groupScan) {
+try {
+
+  DrillRelOptUtil.ProjectPushInfo columnInfo =
+  DrillRelOptUtil.getFieldsInformation(scan.getRowType(), 
project.getProjects());
+  if (columnInfo == null || Utilities.isStarQuery(columnInfo.getFields()) 
//
+  || !groupScan.canPushdownProjects(columnInfo.getFields())) {
+return;
+  }
+  RelTraitSet newTraits = call.getPlanner().emptyTraitSet();
+  // Clear out collation trait
+  for (RelTrait trait : scan.getTraitSet()) {
+if (!(trait instanceof RelCollation)) {
+  newTraits.plus(trait);
+}
+  }
+  final ScanPrel newScan = new ScanPrel(scan.getCluster(), 
newTraits.plus(Prel.DRILL_PHYSICAL),
+  

[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228345668
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
 ##
 @@ -29,6 +29,8 @@
 import org.apache.drill.exec.record.RecordBatch;
 
 public interface Partitioner {
+  int DEFAULT_RECORD_BATCH_SIZE = (1 << 10) - 1;
 
 Review comment:
   I think that was an omission.  I have re-inserted the comment but with some 
modifications because the batch-sizing project now allows operators to set the 
output batch size in terms of Mbytes rather than `recordCount`.  It is not yet 
applied across the board, so this `DEFAULT_RECORD_BATCH_SIZE` is still relevant 
for the exchange operators. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228343223
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
 ##
 @@ -0,0 +1,90 @@
+/*
+ * 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.cost;
+
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMdSelectivity;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+
+import java.util.List;
+
+public class DrillRelMdSelectivity extends RelMdSelectivity {
+  private static final DrillRelMdSelectivity INSTANCE = new 
DrillRelMdSelectivity();
+
+  public static final RelMetadataProvider SOURCE = 
ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.SELECTIVITY.method,
 INSTANCE);
+
+
+  public Double getSelectivity(RelNode rel, RexNode predicate) {
 
 Review comment:
   There were customizations needed for RelSubset, DrillScanRel and ScanPrel 
selectivities.  For all other nodes, we are calling the 
`super.getSelectivity()`. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228342002
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
 ##
 @@ -535,7 +535,10 @@ public FragmentContext getContext() {
 
   @Override
   public BatchSchema getSchema() {
-return outgoingContainer.getSchema();
+if (outgoingContainer.hasSchema()) {
+  return outgoingContainer.getSchema();
+}
+return null;
 
 Review comment:
   The null value could occur for example when even though the 
`VectorContainer` has vectors but the operator has not yet called 
`buildSchema()` on the container.  There's a `Preconditions.checkNotNull` in 
`VectorContainer.getSchema()`. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228337517
 
 

 ##
 File path: pom.xml
 ##
 @@ -53,8 +53,8 @@
 2.9.5
 2.9.5
 3.4.12
-5.2.1-mapr
-1.1
+6.0.1-mapr
 
 Review comment:
   For this version update, I will let the the PR you referenced [1] handle it. 
 There are functional test failures with that PR that need to be addressed. 
   
   [1] https://github.com/apache/drill/pull/1489


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228229608
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,168 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
+
+  final private IndexDescriptor indexDesc;
+
+  private boolean hasFunctionalField = false;
+
+  //when we scan schemaPath in groupscan's columns, we check if this 
column(schemaPath) should be rewritten to '$N',
+  //When there are more than two functions on the same column in index, 
CAST(a.b as INT), CAST(a.b as VARCHAR),
+  // then we should map SchemaPath a.b to a set of SchemaPath, e.g. $1, $2
+  private Map> columnToConvert;
+
+  // map of functional index expression to destination SchemaPath e.g. $N
+  private Map exprToConvert;
+
+  //map of SchemaPath involved in a functional field
+  private Map> pathsInExpr;
+
+  private Set newPathsForIndexedFunction;
+
+  private Set allPathsInFunction;
+
+  public MapRDBFunctionalIndexInfo(IndexDescriptor indexDesc) {
+this.indexDesc = indexDesc;
+columnToConvert = Maps.newHashMap();
 
 Review comment:
   Ah, that explains it.  Presumably we will fix it for 1.15.  Thanks. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-25 Thread GitBox
amansinha100 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_r228055114
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,168 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
+
+  final private IndexDescriptor indexDesc;
+
+  private boolean hasFunctionalField = false;
+
+  //when we scan schemaPath in groupscan's columns, we check if this 
column(schemaPath) should be rewritten to '$N',
+  //When there are more than two functions on the same column in index, 
CAST(a.b as INT), CAST(a.b as VARCHAR),
+  // then we should map SchemaPath a.b to a set of SchemaPath, e.g. $1, $2
+  private Map> columnToConvert;
+
+  // map of functional index expression to destination SchemaPath e.g. $N
+  private Map exprToConvert;
+
+  //map of SchemaPath involved in a functional field
+  private Map> pathsInExpr;
+
+  private Set newPathsForIndexedFunction;
+
+  private Set allPathsInFunction;
+
+  public MapRDBFunctionalIndexInfo(IndexDescriptor indexDesc) {
+this.indexDesc = indexDesc;
+columnToConvert = Maps.newHashMap();
 
 Review comment:
   The checkstyle-validation had identified a bunch of non-shaded Guava classes 
in the core classes when I created the PR and I had fixed those by using the 
shaded version. But oddly, it did not complain for the MapR-DB plugin classes. 
Does this enforcement not occur for the plugins ? I am not sure..but in any 
case I looked through the new code added in MapR-DB plugin and have modified 
the imports to use shaded Guava classes.


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-24 Thread GitBox
amansinha100 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_r228020096
 
 

 ##
 File path: 
protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
 ##
 @@ -78,7 +78,8 @@
 SEQUENCE_SUB_SCAN(53),
 PARTITION_LIMIT(54),
 PCAPNG_SUB_SCAN(55),
-RUNTIME_FILTER(56);
+RUNTIME_FILTER(56),
+ROWKEY_JOIN(57);
 
 Review comment:
   I have regenerated the protobuf files for the native C++ client.  (As an 
aside, the steps for doing this seems complicated unlike the Java version. In 
either case, we really should not need to rebuild the client side if a new 
operator has been added on the server side). 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-19 Thread GitBox
amansinha100 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_r226787783
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
 ##
 @@ -29,6 +29,8 @@
 import org.apache.drill.exec.record.RecordBatch;
 
 public interface Partitioner {
+  int DEFAULT_RECORD_BATCH_SIZE = (1 << 10) - 1;
 
 Review comment:
   The DEFAULT_RECORD_BATCH_SIZE is already in the current master branch here 
[1].  I simply moved it into the `Partitioner` interface. 
   
   [1] 
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java#L58


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-16 Thread GitBox
amansinha100 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_r225740876
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
 ##
 @@ -166,4 +169,25 @@ public void onMatch(RelOptRuleCall call) {
 return list;
   }
 
+  public static Project replace(Project topProject, Project bottomProject) {
 
 Review comment:
   They can be merged into the top level Project that allows 
duplicates...however, the purpose of the `replace()` method here is to simply 
allow the caller to replace a project with another with the assumption that 
callers know exactly what they are doing.  This is not applying the full 
fledged `DrillMergeProjectRule`. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-16 Thread GitBox
amansinha100 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_r225737406
 
 

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
 ##
 @@ -0,0 +1,1715 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.index;
+
+import com.mapr.db.Admin;
+import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
+import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
+import com.mapr.tests.annotations.ClusterTest;
+import org.apache.drill.PlanTestBase;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.apache.drill.common.config.DrillConfig;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.MethodSorters;
+import java.util.Properties;
+
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@Category(ClusterTest.class)
+public class IndexPlanTest extends BaseJsonTest {
+
+  final static String PRIMARY_TABLE_NAME = "/tmp/index_test_primary";
+
+  final static int PRIMARY_TABLE_SIZE = 1;
+  private static final String sliceTargetSmall = "alter session set 
`planner.slice_target` = 1";
+  private static final String sliceTargetDefault = "alter session reset 
`planner.slice_target`";
+  private static final String noIndexPlan = "alter session set 
`planner.enable_index_planning` = false";
+  private static final String defaultHavingIndexPlan = "alter session reset 
`planner.enable_index_planning`";
+  private static final String disableHashAgg = "alter session set 
`planner.enable_hashagg` = false";
+  private static final String enableHashAgg =  "alter session set 
`planner.enable_hashagg` = true";
+  private static final String defaultnonCoveringSelectivityThreshold = "alter 
session set `planner.index.noncovering_selectivity_threshold` = 0.025";
+  private static final String incrnonCoveringSelectivityThreshold = "alter 
session set `planner.index.noncovering_selectivity_threshold` = 0.25";
+  private static final String disableFTS = "alter session set 
`planner.disable_full_table_scan` = true";
+  private static final String enableFTS = "alter session reset 
`planner.disable_full_table_scan`";
+  private static final String preferIntersectPlans = "alter session set 
`planner.index.prefer_intersect_plans` = true";
+  private static final String defaultIntersectPlans = "alter session reset 
`planner.index.prefer_intersect_plans`";
+  private static final String lowRowKeyJoinBackIOFactor
+  = "alter session set `planner.index.rowkeyjoin_cost_factor` = 0.01";
+  private static final String defaultRowKeyJoinBackIOFactor
+  = "alter session reset `planner.index.rowkeyjoin_cost_factor`";
+
+  /**
+   *  A sample row of this 10K table:
+   --+-++
+   | 1012  | {"city":"pfrrs","state":"pc"}  | 
{"email":"kffzkuz...@gmail.com","phone":"655471"}  |
+   {"ssn":"17423"}  | {"fname":"KfFzK","lname":"UZwNk"}  | 
{"age":53.0,"income":45.0}  | 1012   |
+   *
+   * This test suite generate random content to fill all the rows, since the 
random function always start from
+   * the same seed for different runs, when the row count is not changed, the 
data in table will always be the same,
+   * thus the query result could be predicted and verified.
+   */
+
+  @BeforeClass
+  public static void setupTableIndexes() throws Exception {
+
+Properties overrideProps = new Properties();
+
overrideProps.setProperty("format-maprdb.json.useNumRegionsForDistribution", 
"true");
+updateTestCluster(1, DrillConfig.create(overrideProps));
+
+MaprDBTestsSuite.setupTests();
+MaprDBTestsSuite.createPluginAndGetConf(getDrillbitContext());
+
+test(incrnonCoveringSelectivityThreshold);
+
+System.out.print("setupTableIndexes begins");
+Admin admin = MaprDBTestsSuite.getAdmin();
+if (admin != null) {
+  if (admin.tableExists(PRIMARY_TABLE_NAME)) {
+

[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-16 Thread GitBox
amansinha100 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_r225721627
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
 ##
 @@ -0,0 +1,53 @@
+/*
+ * 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.common;
+
+/**
+ * For the int type control,
+ * the meaning of each bit start from lowest:
+ * bit 0: intersect or not, 0 -- default(no intersect), 1 -- INTERSECT 
(DISTINCT as default)
+ * bit 1: intersect type, 0 -- default (DISTINCT), 1 -- INTERSECT_ALL
+ */
+public class JoinControl {
 
 Review comment:
   @Ben-Zvi I have incorporated the joinControl logic as part of the hash join 
probe phase.  Please see changes in [1].  I haven't done it for build phase 
since it will be superseded by the semi-join changes.  Can you pls review this 
change ?
   
   [1] 
https://github.com/apache/drill/pull/1466/commits/5330fd684a20587e9e860d7894e3d3602a6d0495


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224928601
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
 ##
 @@ -517,6 +388,15 @@ private static FieldPath 
getFieldPathForProjection(SchemaPath column) {
 return new FieldPath(child);
   }
 
+  public static boolean includesIdField(Collection projected) {
+return Iterables.tryFind(projected, new Predicate() {
 
 Review comment:
   Since this is a syntactic modification, I am inclined to leave it as-is .. 
partly because I might cause an unintentional side effect/regression and partly 
because such functional utilities appear elsewhere in the Drill and Calcite 
code.  Ideally, if we want to move to Lambdas, we should re-visit all such 
implementations.  


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224926241
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
 ##
 @@ -137,11 +137,10 @@ protected void 
doPushFilterIntoJsonGroupScan(RelOptRuleCall call,
   return; //no filter pushdown ==> No transformation.
 }
 
-// clone the groupScan with the newScanSpec.
-final JsonTableGroupScan newGroupsScan = groupScan.clone(newScanSpec);
+final JsonTableGroupScan newGroupsScan = (JsonTableGroupScan) 
groupScan.clone(newScanSpec);
 
 Review comment:
   Currently, the `clone()` method returns an instance of `GroupScan`, so this 
cast is needed. Did you mean change the original `clone` implementation itself 
?  That will require some additional changes since both binary table and json 
table implement it. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224925146
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/IndexIntersectPlanGenerator.java
 ##
 @@ -0,0 +1,350 @@
+/*
+ * 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.generators;
+
+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.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.JoinControl;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.FunctionalIndexHelper;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.index.IndexConditionInfo;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
+import 
org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionType;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.HashJoinPrel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * IndexScanIntersectGenerator is to generate index plan against multiple 
index tables,
+ * the input indexes are assumed to be ranked by selectivity(low to high) 
already.
+ */
+public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(IndexIntersectPlanGenerator.class);
+
+  final Map indexInfoMap;
+
+  public IndexIntersectPlanGenerator(IndexLogicalPlanCallContext indexContext,
+ Map 
indexInfoMap,
+ RexBuilder builder,
+ PlannerSettings settings) {
+super(indexContext, null, null, builder, settings);
+this.indexInfoMap = indexInfoMap;
+  }
+
+  public RelNode buildRowKeyJoin(RelNode left, RelNode right, boolean 
isRowKeyJoin, int htControl)
+  throws InvalidRelException {
+final int leftRowKeyIdx = getRowKeyIndex(left.getRowType(), origScan);
+final int rightRowKeyIdx = 0; // only rowkey field is being projected from 
right side
+
+assert leftRowKeyIdx >= 0;
+
+List leftJoinKeys = ImmutableList.of(leftRowKeyIdx);
+List rightJoinKeys = ImmutableList.of(rightRowKeyIdx);
+
+logger.trace(String.format(
+"buildRowKeyJoin: leftIdx: %d, rightIdx: %d",
+leftRowKeyIdx, rightRowKeyIdx));
+RexNode joinCondition =
+RelOptUtil.createEquiJoinCondition(left, leftJoinKeys,
+right, rightJoinKeys, builder);
+
+if (isRowKeyJoin == true) {
 
 Review comment:
   done


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224925023
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
 ##
 @@ -114,6 +114,28 @@
   public static final String UNIONALL_DISTRIBUTE_KEY = 
"planner.enable_unionall_distribute";
   public static final BooleanValidator UNIONALL_DISTRIBUTE = new 
BooleanValidator(UNIONALL_DISTRIBUTE_KEY, null);
 
+  // --- Index planning related 
options BEGIN --
+  public static final String USE_SIMPLE_OPTIMIZER_KEY = 
"planner.use_simple_optimizer";
+  public static final BooleanValidator USE_SIMPLE_OPTIMIZER = new 
BooleanValidator(USE_SIMPLE_OPTIMIZER_KEY, null);
+  public static final BooleanValidator INDEX_PLANNING = new 
BooleanValidator("planner.enable_index_planning", null);
+  public static final BooleanValidator ENABLE_STATS = new 
BooleanValidator("planner.enable_statistics", null);
 
 Review comment:
   I will let @gparai elaborate on this but my understanding is that by keeping 
a general name such as `planner.enable_statistics` option we could potentially 
use it for any type of data source (including Parquet) once the broader stats 
capability is added to Drill. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224921473
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
 ##
 @@ -0,0 +1,27 @@
+/*
+ * 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 org.apache.drill.common.exceptions.DrillRuntimeException;
+
+public class InvalidIndexDefinitionException extends DrillRuntimeException {
 
 Review comment:
   Done


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224921316
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
 ##
 @@ -57,34 +64,74 @@ public MapRDBFormatPlugin(String name, DrillbitContext 
context, Configuration fs
 hbaseConf = HBaseConfiguration.create(fsConf);
 hbaseConf.set(ConnectionFactory.DEFAULT_DB, 
ConnectionFactory.MAPR_ENGINE2);
 connection = ConnectionFactory.createConnection(hbaseConf);
+jsonTableCache = new MapRDBTableCache(context.getConfig());
+int scanRangeSizeMBConfig = 
context.getConfig().getInt(PluginConstants.JSON_TABLE_SCAN_SIZE_MB);
+if (scanRangeSizeMBConfig < 32 || scanRangeSizeMBConfig > 8192) {
 
 Review comment:
   Done


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224918663
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
 ##
 @@ -187,7 +186,7 @@ protected void doPushFilterIntoBinaryGroupScan(final 
RelOptRuleCall call,
 
groupScan.getTableStats());
 newGroupsScan.setFilterPushedDown(true);
 
-final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), 
newGroupsScan, scan.getRowType());
+final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), 
newGroupsScan, scan.getRowType(), scan.getTable());
 
 Review comment:
   Changed to use the constructor.  @vdiravka why not remove the `create` 
method altogether ?   [Updated] never mind...I see that the latest master has 
that removed.  


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224918663
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
 ##
 @@ -187,7 +186,7 @@ protected void doPushFilterIntoBinaryGroupScan(final 
RelOptRuleCall call,
 
groupScan.getTableStats());
 newGroupsScan.setFilterPushedDown(true);
 
-final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), 
newGroupsScan, scan.getRowType());
+final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), 
newGroupsScan, scan.getRowType(), scan.getTable());
 
 Review comment:
   Changed to use the constructor.  @vdiravka why not remove the `create` 
method altogether ?


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224917712
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
 ##
 @@ -0,0 +1,145 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.util.Utilities;
+
+import java.util.List;
+
+public abstract class MapRDBPushProjectIntoScan extends 
StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(MapRDBPushProjectIntoScan.class);
+
+  private MapRDBPushProjectIntoScan(RelOptRuleOperand operand, String 
description) {
+super(operand, description);
+  }
+
+  public static final StoragePluginOptimizerRule PROJECT_ON_SCAN = new 
MapRDBPushProjectIntoScan(
+  RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class)), 
"MapRDBPushProjIntoScan:Proj_On_Scan") {
+@Override
+public void onMatch(RelOptRuleCall call) {
+  final ScanPrel scan = (ScanPrel) call.rel(1);
+  final ProjectPrel project = (ProjectPrel) call.rel(0);
+  if (!(scan.getGroupScan() instanceof MapRDBGroupScan)) {
+return;
+  }
+  doPushProjectIntoGroupScan(call, project, scan, (MapRDBGroupScan) 
scan.getGroupScan());
+  if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
 
 Review comment:
   I have made the changes here and rest of this class to only check for 
JsonTableGroupScan.  Binary table's projection pushdown was already handled by 
the logical planning rule `DrillPushProjectIntoScan`.   In future, if we want 
to extend this rule to binary tables, we would need to ensure it is tested. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-12 Thread GitBox
amansinha100 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_r224916314
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.store.mapr.db;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.LimitPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.hbase.HBaseScanSpec;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.RestrictedJsonTableGroupScan;
+
+public abstract class MapRDBPushLimitIntoScan extends 
StoragePluginOptimizerRule {
 
 Review comment:
   The `DrillPushLimitToScanRule` works for the logical plan.  We introduced 
the MapR-DB specific rule to apply similar pushdown for the physical plan 
because after index planning is done additional pushdowns may be possible. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-10 Thread GitBox
amansinha100 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_r224287561
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
 ##
 @@ -22,7 +22,7 @@
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.calcite.rel.type.RelDataType;
 
 Review comment:
   It probably does..can you create a JIRA for the renaming since it is 
independent of this PR. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-10 Thread GitBox
amansinha100 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_r224286776
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
 ##
 @@ -0,0 +1,145 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.util.Utilities;
+
+import java.util.List;
+
+public abstract class MapRDBPushProjectIntoScan extends 
StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(MapRDBPushProjectIntoScan.class);
+
+  private MapRDBPushProjectIntoScan(RelOptRuleOperand operand, String 
description) {
+super(operand, description);
+  }
+
+  public static final StoragePluginOptimizerRule PROJECT_ON_SCAN = new 
MapRDBPushProjectIntoScan(
+  RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class)), 
"MapRDBPushProjIntoScan:Proj_On_Scan") {
+@Override
+public void onMatch(RelOptRuleCall call) {
+  final ScanPrel scan = (ScanPrel) call.rel(1);
+  final ProjectPrel project = (ProjectPrel) call.rel(0);
+  if (!(scan.getGroupScan() instanceof MapRDBGroupScan)) {
+return;
+  }
+  doPushProjectIntoGroupScan(call, project, scan, (MapRDBGroupScan) 
scan.getGroupScan());
+  if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
 
 Review comment:
   Good catch..this pushdown was only intended for `JsonTableGroupScan`, so not 
sure why that extra call before the `if` block is there.  Will look into the 
origins of it. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-10 Thread GitBox
amansinha100 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_r224282950
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
 ##
 @@ -0,0 +1,145 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.util.Utilities;
+
+import java.util.List;
+
+public abstract class MapRDBPushProjectIntoScan extends 
StoragePluginOptimizerRule {
 
 Review comment:
   The main difference is that DrillPushProjectIntoScanRule is applied during 
logical planning phase whereas the MapRDBPushProjectIntoScan is applied during 
physical planning because we want to ensure that even after new physical plans 
are created (such as index based plans), the projection pushdown is done. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-10 Thread GitBox
amansinha100 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_r224281194
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,168 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
+
+  final private IndexDescriptor indexDesc;
+
+  private boolean hasFunctionalField = false;
+
+  //when we scan schemaPath in groupscan's columns, we check if this 
column(schemaPath) should be rewritten to '$N',
+  //When there are more than two functions on the same column in index, 
CAST(a.b as INT), CAST(a.b as VARCHAR),
+  // then we should map SchemaPath a.b to a set of SchemaPath, e.g. $1, $2
+  private Map> columnToConvert;
+
+  // map of functional index expression to destination SchemaPath e.g. $N
+  private Map exprToConvert;
+
+  //map of SchemaPath involved in a functional field
+  private Map> pathsInExpr;
+
+  private Set newPathsForIndexedFunction;
+
+  private Set allPathsInFunction;
+
+  public MapRDBFunctionalIndexInfo(IndexDescriptor indexDesc) {
+this.indexDesc = indexDesc;
+columnToConvert = Maps.newHashMap();
 
 Review comment:
   I am inclined to leave these as-is since there are quite a few places where 
we use the Google commons package to create not just HashMaps but Lists, Sets 
and other collections. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-10 Thread GitBox
amansinha100 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_r224278895
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.logical.DrillTable;
+
+import java.io.IOException;
+import java.util.List;
+
+public class DrillIndexDescriptor extends AbstractIndexDescriptor {
+
+  /**
+   * The name of Drill's Storage Plugin on which the Index was stored
+   */
+  private String storage;
+
+  private DrillTable table;
+
+  public DrillIndexDescriptor(List indexCols,
+  CollationContext indexCollationContext,
+  List nonIndexCols,
+  List rowKeyColumns,
+  String indexName,
+  String tableName,
+  IndexType type,
+  NullDirection nullsDirection) {
+super(indexCols, indexCollationContext, nonIndexCols, rowKeyColumns, 
indexName, tableName, type, nullsDirection);
+  }
+
+  public DrillIndexDescriptor(DrillIndexDefinition def) {
+this(def.indexColumns, def.indexCollationContext, def.nonIndexColumns, 
def.rowKeyColumns, def.indexName,
+def.getTableName(), def.getIndexType(), def.nullsDirection);
+  }
+
+  @Override
+  public double getRows(RelNode scan, RexNode indexCondition) {
+//TODO: real implementation is to use Drill's stats implementation. for 
now return fake value 1.0
+return 1.0;
+  }
+
+  @Override
+  public IndexGroupScan getIndexGroupScan() {
+try {
+  final DrillTable idxTable = getDrillTable();
+  GroupScan scan = idxTable.getGroupScan();
+
+  if (!(scan instanceof IndexGroupScan)){
+logger.error("The Groupscan from table {} is not an IndexGroupScan", 
idxTable.toString());
+return null;
+  }
+  return (IndexGroupScan)scan;
+}
+catch(IOException e) {
+  logger.error("Error in getIndexGroupScan ", e);
+}
+return null;
+  }
+
+  public void attach(String storageName, DrillTable inTable) {
 
 Review comment:
   Removed attach() since it was not being used.  Added Javadoc or `@Override` 
for few others. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-09 Thread GitBox
amansinha100 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_r223878408
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
 ##
 @@ -517,6 +388,15 @@ private static FieldPath 
getFieldPathForProjection(SchemaPath column) {
 return new FieldPath(child);
   }
 
+  public static boolean includesIdField(Collection projected) {
+return Iterables.tryFind(projected, new Predicate() {
 
 Review comment:
   @vdiravka could you elaborate on this a bit ?  I haven't done much with 
lambda expressions in Java 8, but if you show how to rewrite this statement and 
assuming it has an advantage over the existing implementation, I would be happy 
to.  Thanks. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-10-01 Thread GitBox
amansinha100 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_r221718343
 
 

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
 ##
 @@ -0,0 +1,219 @@
+/*
+ * 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.store.mapr.db;
+
+import com.mapr.db.impl.IdCodec;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+import org.apache.drill.exec.record.AbstractRecordBatch.BatchState;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A RestrictedMapRDBSubScanSpec encapsulates a join instance which contains 
the ValueVectors of row keys and
+ * is associated with this sub-scan and also exposes an iterator type 
interface over the row key vectors.
+ */
+public class RestrictedMapRDBSubScanSpec extends MapRDBSubScanSpec {
+
+  /**
+   * The RowKeyJoin instance (specific to one minor fragment) which will 
supply this
+   * subscan with the set of rowkeys. For efficiency, we keep a reference to 
this
+   * join rather than making another copy of the rowkeys.
+   */
+  private RowKeyJoin rjbatch = null;
+
+  /**
+   * The following are needed to maintain internal state of iteration over the 
set
+   * of row keys
+   */
+  private ValueVector rowKeyVector = null; // the current row key value vector
+  private int currentIndex = 0;  // the current index within the row key vector
+  private int maxOccupiedIndex = -1; // max occupied index within a row key 
vector
+
+  public RestrictedMapRDBSubScanSpec(String tableName, String regionServer, 
byte[] serializedFilter, String userName) {
+super(tableName, null, regionServer, null, null, serializedFilter, null, 
userName);
+  }
+  /* package */ RestrictedMapRDBSubScanSpec() {
+// empty constructor, to be used with builder pattern;
+  }
+
+  public void setJoinForSubScan(RowKeyJoin rjbatch) {
+this.rjbatch = rjbatch;
+  }
+
+  @JsonIgnore
+  public RowKeyJoin getJoinForSubScan() {
+return rjbatch;
+  }
+
+  @JsonIgnore
+  private void init(Pair b) {
+this.maxOccupiedIndex = b.getRight();
+this.rowKeyVector = b.getLeft();
+this.currentIndex = 0;
+  }
+
+  /**
+   * Return {@code true} if a valid rowkey batch is available, {@code false} 
otherwise
+   */
+  @JsonIgnore
+  public boolean readyToGetRowKey() {
+return rjbatch != null && rjbatch.hasRowKeyBatch();
+  }
+
+  /**
+   * Return {@code true} if the row key join is in the build schema phase
+   */
+  @JsonIgnore
+  public boolean isBuildSchemaPhase() {
+return rjbatch.getBatchState() == BatchState.BUILD_SCHEMA;
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more row keys.
+   * (In other words, returns {@code true} if {@link #nextRowKey} would
+   * return a non-null row key)
+   * @return {@code true} if the iteration has more row keys
+   */
+  @JsonIgnore
+  public boolean hasRowKey() {
+if (rowKeyVector != null && currentIndex <= maxOccupiedIndex) {
+  return true;
+}
+
+if (rjbatch != null) {
+  Pair currentBatch = rjbatch.nextRowKeyBatch();
+
+  // note that the hash table could be null initially during the 
BUILD_SCHEMA phase
+  if (currentBatch != null) {
+init(currentBatch);
+return true;
+  }
+}
+
+return false;
+  }
+
+  @JsonIgnore
+  public int getMaxRowKeysToBeRead() {
+if (rjbatch != null) {
+  Pair currentBatch = rjbatch.nextRowKeyBatch();
+
+  // note that the currentBatch could be null initially during the 
BUILD_SCHEMA phase
+  if (currentBatch != null) {
+init(currentBatch);
+  }
+}
+return maxOccupiedIndex + 1;
+  }
+
+  /**
+   * Returns number of rowKeys that can be read.
+   * Number of rowKeys returned will be numRowKeysToRead at the most i.e. it
+   * will be less than numRowKeysToRead if only that many exist in the 
currentBatch.
+   */
+  @JsonIgnore
+  public 

[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-21 Thread GitBox
amansinha100 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_r219396438
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
 ##
 @@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import java.util.List;
+
+/**
+ * Encapsulates one or more IndexProperties representing (non)covering or 
intersecting indexes. The encapsulated
+ * IndexProperties are used to rank the index in comparison with other 
IndexGroups.
+ */
+public class IndexGroup {
+  private List indexProps;
+
+  public IndexGroup() {
+indexProps = Lists.newArrayList();
+  }
+
+  public boolean isIntersectIndex() {
+if (indexProps.size() > 1) {
+  return true;
+} else {
+  return false;
+}
 
 Review comment:
   Agreed.  Done.


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-21 Thread GitBox
amansinha100 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_r219395866
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
 ##
 @@ -0,0 +1,53 @@
+/*
+ * 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.common;
+
+/**
+ * For the int type control,
+ * the meaning of each bit start from lowest:
+ * bit 0: intersect or not, 0 -- default(no intersect), 1 -- INTERSECT 
(DISTINCT as default)
+ * bit 1: intersect type, 0 -- default (DISTINCT), 1 -- INTERSECT_ALL
+ */
+public class JoinControl {
 
 Review comment:
   Hmm...it seems a small part of the code in HashJoinBatch may not have been 
merged correctly .. either when I did the recent merge or when previously we 
merged the index code with the hash join spilling code.  On a separate private 
branch, I do see `JoinControl` being used for a decision point.  I will talk to 
you offline about this. 
   
   I am also tagging @chunhui-shi who is a co-author and had added the 
`JoinControl` specifically to distinguish between INTERSECT_DISTINCT and 
INTERSECT_ALL.  


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-19 Thread GitBox
amansinha100 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_r219033777
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
 ##
 @@ -86,7 +86,10 @@ public RelNode convert(
 return new HashToRandomExchangePrel(rel.getCluster(), 
planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel,
  toDist.getFields());
   case RANGE_DISTRIBUTED:
-return new OrderedPartitionExchangePrel(rel.getCluster(), 
planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
+// return new OrderedPartitionExchangePrel(rel.getCluster(), 
planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
 
 Review comment:
   The OrderedPartitionExchange was not being used anywhere.  Its original 
purpose was to do range-based sorting; however this was not completely 
functional.  It was also a much more complex operator (see its implementation 
in OrderedPartitionRecordBatch). I think if we go back to fixing it and using 
it in future, we will probably add a new type of distribution trait.  I will 
add explanation along these lines. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-19 Thread GitBox
amansinha100 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_r219032715
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.physical;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
+import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+public class RowKeyJoinPrel extends JoinPrel implements Prel {
+
+  double estimatedRowCount = -1;
+  public RowKeyJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode 
left, RelNode right,
+  RexNode condition, JoinRelType joinType) throws InvalidRelException {
+super(cluster, traits, left, right, condition, joinType);
+Preconditions.checkArgument(joinType == JoinRelType.INNER);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) 
throws IOException {
+PhysicalOperator leftPop = ((Prel)left).getPhysicalOperator(creator);
+PhysicalOperator rightPop = ((Prel)right).getPhysicalOperator(creator);
+RowKeyJoinPOP rkPop = new RowKeyJoinPOP(leftPop, rightPop);
+return creator.addMetadata(this, rkPop);
+  }
+
+  @Override
+  public double estimateRowCount(RelMetadataQuery mq) {
+if (estimatedRowCount >= 0) {
+  return estimatedRowCount;
+}
+return this.getLeft().getRows();
+  }
+
+  @Override
+  public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, 
RelNode right,
+  JoinRelType joinType, boolean semiJoinDone) {
+try {
+  RowKeyJoinPrel rkj = new RowKeyJoinPrel(this.getCluster(), traitSet, 
left, right, conditionExpr, joinType);
+  rkj.setEstimatedRowCount(this.estimatedRowCount);
+  return rkj;
+} catch (InvalidRelException e) {
+  throw new AssertionError(e);
+}
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery 
mq) {
+if(PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
+  return super.computeSelfCost(planner).multiplyBy(.1);
+}
+double rowCount = mq.getRowCount(this.getRight());
+DrillCostFactory costFactory = (DrillCostFactory) planner.getCostFactory();
+return costFactory.makeCost(rowCount, 0, 0, 0,
 
 Review comment:
   RowKeyJoin operator by itself incurs negligible CPU and I/O cost since it is 
not doing a real join.  The actual cost is attributed to the skip-scan (random 
I/O).  The RK join will hold 1 batch in memory but since it is not a buffering 
operator, we typically don't assign memory costs to them.   I can add the 
explanation. 


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


With regards,
Apache Git Services


[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-19 Thread GitBox
amansinha100 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_r219032311
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/NonCoveringIndexPlanGenerator.java
 ##
 @@ -0,0 +1,335 @@
+/*
+ * 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.generators;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.JoinControl;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.FunctionalIndexHelper;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.HashJoinPrel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ * Generate a non-covering index plan that is equivalent to the original plan. 
The non-covering plan consists
+ * of a join-back between an index lookup and the primary table. This 
join-back is performed using a rowkey join.
+ * For the primary table, we use a restricted scan that allows doing skip-scan 
instead of sequential scan.
+ *
+ * Original Plan:
+ *   Filter
+ * |
+ *DBGroupScan
+ *
+ * New Plan:
+ *
+ *RowKeyJoin
+ *  / \
+ * Remainder Filter  Exchange
+ * ||
+ *   RestrictedFilter (with index columns only)
+ *   DBGroupScan|
+ *  IndexGroupScan
+ *
+ * This plan will be further optimized by the filter pushdown rule of the 
Index plugin which should
+ * push the index column filters into the index scan.
+ */
+public class NonCoveringIndexPlanGenerator extends AbstractIndexPlanGenerator {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(NonCoveringIndexPlanGenerator.class);
+  final protected IndexGroupScan indexGroupScan;
+  final private IndexDescriptor indexDesc;
+  // Ideally This functionInfo should be cached along with indexDesc.
+  final protected FunctionalIndexInfo functionInfo;
+
+  public NonCoveringIndexPlanGenerator(IndexLogicalPlanCallContext 
indexContext,
+   IndexDescriptor indexDesc,
+   IndexGroupScan indexGroupScan,
+   RexNode indexCondition,
+ 

[GitHub] amansinha100 commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

2018-09-15 Thread GitBox
amansinha100 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_r217884619
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
 ##
 @@ -0,0 +1,53 @@
+/*
+ * 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.common;
+
+/**
+ * For the int type control,
+ * the meaning of each bit start from lowest:
+ * bit 0: intersect or not, 0 -- default(no intersect), 1 -- INTERSECT 
(DISTINCT as default)
+ * bit 1: intersect type, 0 -- default (DISTINCT), 1 -- INTERSECT_ALL
+ */
+public class JoinControl {
 
 Review comment:
   The JoinControl is actually used in IndexIntersectPlanGenerator, 
NonCoveringIndexPlanGenerator and a few other places.  You may need to expand 
those files since GitHub does not automatically load the diffs by default. 
   
   For the broader question of passing around the join type and related 
information in a more componentized way, yes this has been a pending item that 
planner needs to provide.  Pls feel free to create a JIRA and if you have any 
initial thoughts you can start proposing it there. 


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


With regards,
Apache Git Services