[GitHub] paul-rogers opened a new pull request #1501: DRILL-6791: Scan projection framework

2018-10-11 Thread GitBox
paul-rogers opened a new pull request #1501: DRILL-6791: Scan projection 
framework
URL: https://github.com/apache/drill/pull/1501
 
 
   The "schema projection" mechanism:
   
   * Handles none (SELECT COUNT\(*)), some (SELECT a, b, x) and all (SELECT *) 
projection.
   * Handles null columns (for projection a column "x" that does not exist in 
the base table.)
   * Handles constant columns as used for file metadata (AKA "implicit" 
columns).
   * Handle schema persistence: the need to reuse the same vectors across 
different scanners
   * Provides a framework for consuming externally-supplied metadata
   * Since we don't yet have a way to provide "real" metadata, obtains metadata 
hints from previous batches and from the projection list (a.b implies that "a" 
is a map, c[0] implies that "c" is an array, etc.)
   * Handles merging the set of data source columns and null columns to create 
the final output batch.


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
 ##
 @@ -0,0 +1,250 @@
+/*
+ * 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.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import 
org.apache.drill.exec.planner.logical.partition.RewriteCombineBinaryOperators;
+import org.apache.calcite.rel.RelNode;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class IndexConditionInfo {
+  public final RexNode indexCondition;
+  public final RexNode remainderCondition;
+  public final boolean hasIndexCol;
+
+  public IndexConditionInfo(RexNode indexCondition, RexNode 
remainderCondition, boolean hasIndexCol) {
+this.indexCondition = indexCondition;
+this.remainderCondition = remainderCondition;
+this.hasIndexCol = hasIndexCol;
+  }
+
+  public static Builder newBuilder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan) {
+return new Builder(condition, indexes, builder, scan);
+  }
+
+  public static class Builder {
+final RexBuilder builder;
+final RelNode scan;
+final Iterable indexes;
+private RexNode condition;
+
+public Builder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan
+) {
+  this.condition = condition;
+  this.builder = builder;
+  this.scan = scan;
+  this.indexes = indexes;
+}
+
+public Builder(RexNode condition,
+   IndexDescriptor index,
+   RexBuilder builder,
+   DrillScanRel scan
+) {
+  this.condition = condition;
+  this.builder = builder;
+  this.scan = scan;
+  this.indexes = Lists.newArrayList(index);
+}
+
+/**
+ * Get a single IndexConditionInfo in which indexCondition has field  on 
all indexes in this.indexes
+ * @return
+ */
+public IndexConditionInfo getCollectiveInfo(IndexLogicalPlanCallContext 
indexContext) {
+  Set paths = Sets.newLinkedHashSet();
+  for ( IndexDescriptor index : indexes ) {
+paths.addAll(index.getIndexColumns());
+//paths.addAll(index.getNonIndexColumns());
+  }
+  return indexConditionRelatedToFields(Lists.newArrayList(paths), 
condition);
+}
+
+/*
+ * A utility function to check whether the given index hint is valid.
+ */
+public boolean isValidIndexHint(IndexLogicalPlanCallContext indexContext) {
+  if (indexContext.indexHint.equals("")) { return false; }
+
+  for ( IndexDescriptor index: indexes ) {
+if ( indexContext.indexHint.equals(index.getIndexName())) {
+  return true;
+}
+  }
+  return false;
+}
+
+/**
+ * Get a map of Index=>IndexConditionInfo, each IndexConditionInfo has the 
separated condition and remainder condition.
+ * The map is ordered, so the last IndexDescriptor will have the final 
remainderCondition after separating conditions
+ * that are relevant to this.indexes. The conditions are separated on 
LEADING index columns.
+ * @return Map containing index{@link IndexDescriptor} and condition 
{@link IndexConditionInfo} pairs
+ */
+public Map 
getFirstKeyIndexConditionMap() {
+
+  Map indexInfoMap = 
Maps.newLinkedHashMap();

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
 ##
 @@ -0,0 +1,250 @@
+/*
+ * 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.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import 
org.apache.drill.exec.planner.logical.partition.RewriteCombineBinaryOperators;
+import org.apache.calcite.rel.RelNode;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class IndexConditionInfo {
+  public final RexNode indexCondition;
+  public final RexNode remainderCondition;
+  public final boolean hasIndexCol;
+
+  public IndexConditionInfo(RexNode indexCondition, RexNode 
remainderCondition, boolean hasIndexCol) {
+this.indexCondition = indexCondition;
+this.remainderCondition = remainderCondition;
+this.hasIndexCol = hasIndexCol;
+  }
+
+  public static Builder newBuilder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan) {
+return new Builder(condition, indexes, builder, scan);
+  }
+
+  public static class Builder {
+final RexBuilder builder;
+final RelNode scan;
+final Iterable indexes;
+private RexNode condition;
+
+public Builder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan
+) {
+  this.condition = condition;
+  this.builder = builder;
+  this.scan = scan;
+  this.indexes = indexes;
+}
+
+public Builder(RexNode condition,
+   IndexDescriptor index,
+   RexBuilder builder,
+   DrillScanRel scan
+) {
+  this.condition = condition;
+  this.builder = builder;
+  this.scan = scan;
+  this.indexes = Lists.newArrayList(index);
+}
+
+/**
+ * Get a single IndexConditionInfo in which indexCondition has field  on 
all indexes in this.indexes
+ * @return
+ */
+public IndexConditionInfo getCollectiveInfo(IndexLogicalPlanCallContext 
indexContext) {
+  Set paths = Sets.newLinkedHashSet();
+  for ( IndexDescriptor index : indexes ) {
+paths.addAll(index.getIndexColumns());
+//paths.addAll(index.getNonIndexColumns());
+  }
+  return indexConditionRelatedToFields(Lists.newArrayList(paths), 
condition);
+}
+
+/*
+ * A utility function to check whether the given index hint is valid.
+ */
+public boolean isValidIndexHint(IndexLogicalPlanCallContext indexContext) {
+  if (indexContext.indexHint.equals("")) { return false; }
+
+  for ( IndexDescriptor index: indexes ) {
 
 Review comment:
   formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
 ##
 @@ -0,0 +1,176 @@
+/*
+ * 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 static com.mapr.drill.maprdb.tests.MaprDBTestsSuite.INDEX_FLUSH_TIMEOUT;
+
+import java.io.InputStream;
+import java.io.StringBufferInputStream;
+
+import org.apache.hadoop.fs.Path;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
+
+import com.mapr.db.Admin;
+import com.mapr.db.Table;
+import com.mapr.db.TableDescriptor;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.impl.TableDescriptorImpl;
+import com.mapr.db.tests.utils.DBTests;
+import com.mapr.fs.utils.ssh.TestCluster;
+
+/**
+ * This class is to generate a MapR json table of this schema:
+ * {
+ *   "address" : {
+ *  "city":"wtj",
+ *  "state":"ho"
+ *   }
+ *   "contact" : {
+ *  "email":"vcfahj...@gmail.com",
+ *  "phone":"655583"
+ *   }
+ *   "id" : {
+ *  "ssn":"15461"
+ *   }
+ *   "name" : {
+ *  "fname":"VcFahj",
+ *  "lname":"RfM"
+ *   }
+ * }
+ *
+ */
+public class LargeTableGen extends LargeTableGenBase {
+
+  static final int SPLIT_SIZE = 5000;
+  private Admin admin;
+
+  public LargeTableGen(Admin dbadmin) {
+admin = dbadmin;
+  }
+
+  Table createOrGetTable(String tableName, int recordNum) {
+if (admin.tableExists(tableName)) {
+  return MapRDBImpl.getTable(tableName);
+  //admin.deleteTable(tableName);
+}
+else {
+  TableDescriptor desc = new TableDescriptorImpl(new Path(tableName));
+
+  int splits = (recordNum / SPLIT_SIZE) - (((recordNum % SPLIT_SIZE) > 1)? 
0 : 1);
+
+  String[] splitsStr = new String[splits];
+  StringBuilder strBuilder = new StringBuilder("Splits:");
+  for(int i=0; i

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+
+/**
+ * This is a placeholder for the nottypeof() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed 
down
+ * by the storage plug-in into DB. That process will go through 
JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed 
down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
 
 Review comment:
   the same regarding `FunctionTemplate` formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 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:
   You can use it, but from the last Guava version used by Drill (23.0 version) 
there is a note for this factory method:
   > Note for Java 7 and later: this method is now unnecessary and
   > should be treated as deprecated. Instead, use the {@code HashMap}
   > constructor directly, taking advantage of the new
   > http://goo.gl/iz2Wi;>"diamond" syntax.
   
https://google.github.io/guava/releases/snapshot/api/docs/com/google/common/collect/Maps.html#newHashMap--
   
   There was no this note in the previous Guava version used by Drill (19.0 
version), therefore some usages of this method are left.


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
 ##
 @@ -33,7 +33,9 @@
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
-public class MapRDBScanBatchCreator implements BatchCreator {
+public class MapRDBScanBatchCreator implements BatchCreator{
 
 Review comment:
   space before curly brace


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 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) {
+final List newProjects =
+RelOptUtil.pushPastProject(topProject.getProjects(), bottomProject);
+
+// replace the two projects with a combined projection
+if(topProject instanceof DrillProjectRel) {
 
 Review comment:
   space


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/AbstractMatchFunction.java
 ##
 @@ -0,0 +1,58 @@
+/*
+ * 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.rules;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+
+public abstract class AbstractMatchFunction implements MatchFunction {
+  public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  DbGroupScan dbscan = ((DbGroupScan) groupScan);
+  //if we already applied index convert rule, and this scan is indexScan 
or restricted scan already,
+  //no more trying index convert rule
+  return dbscan.supportsSecondaryIndex() && (!dbscan.isIndexScan()) && 
(!dbscan.isRestrictedScan());
+}
+return false;
+  }
+
+  public boolean checkScan(GroupScan groupScan) {
+if (groupScan instanceof DbGroupScan) {
+  DbGroupScan dbscan = ((DbGroupScan) groupScan);
+  //if we already applied index convert rule, and this scan is indexScan 
or restricted scan already,
+  //no more trying index convert rule
+  return dbscan.supportsSecondaryIndex() &&
+ !dbscan.isRestrictedScan() &&
+  (!dbscan.isFilterPushedDown() || dbscan.isIndexScan()) &&
+ !containsStar(dbscan);
+}
+return false;
+  }
+
+  public static boolean containsStar(DbGroupScan dbscan) {
+for (SchemaPath column : dbscan.getColumns()) {
+  if (column.getRootSegment().getPath().startsWith("*")) {
+return true;
+  }
+}
+return false;
+  }
+}
 
 Review comment:
   new line


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/MatchFunction.java
 ##
 @@ -0,0 +1,25 @@
+/*
+ * 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.rules;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+
+public interface MatchFunction {
+  boolean match(RelOptRuleCall call);
+  T onMatch(RelOptRuleCall call);
+}
 
 Review comment:
   new line


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 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) {
+  RelNode newRel;
+  if 

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

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

 ##
 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:
   Agree. I will take a note to do it in context of next Calcite version update 
in 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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/SimpleRexRemap.java
 ##
 @@ -0,0 +1,300 @@
+/*
+ * 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.ImmutableMap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexVisitorImpl;
+
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Rewrite RexNode with these policies:
+ * 1) field renamed. The input field was named differently in index table,
+ * 2) field is in different position of underlying rowtype
+ *
+ * TODO: 3) certain operator needs rewriting. e.g. CAST function
+ * This class for now applies to only filter on scan, for 
filter-on-project-on-scan. A stack of
+ * rowType is required.
+ */
+public class SimpleRexRemap {
+  final RelNode origRel;
+  final RelDataType origRowType;
+  final RelDataType newRowType;
+
+  private RexBuilder builder;
+  private Map destExprMap;
+
+  public SimpleRexRemap(RelNode origRel,
+RelDataType newRowType, RexBuilder builder) {
+super();
+this.origRel = origRel;
+this.origRowType = origRel.getRowType();
+this.newRowType = newRowType;
+this.builder = builder;
+this.destExprMap = Maps.newHashMap();
+  }
+
+  /**
+   * Set the map of src expression to target expression, expressions not in 
the map do not have assigned destinations
+   * @param exprMap
+   * @return
+   */
+  public SimpleRexRemap setExpressionMap(Map  exprMap) {
+destExprMap.putAll(exprMap);
+return this;
+  }
+
+  public RexNode rewriteEqualOnCharToLike(RexNode expr,
+  Map 
equalOnCastCharExprs) {
+Map srcToReplace = Maps.newIdentityHashMap();
+for(Map.Entry entry: 
equalOnCastCharExprs.entrySet()) {
+  RexNode equalOp = entry.getKey();
+  LogicalExpression opInput = entry.getValue();
+
+  final List operands = ((RexCall)equalOp).getOperands();
+  RexLiteral newLiteral = null;
+  RexNode input = null;
+  if(operands.size() == 2 ) {
+RexLiteral oplit = null;
+if (operands.get(0) instanceof RexLiteral) {
+  oplit = (RexLiteral) operands.get(0);
+  if(oplit.getTypeName() == SqlTypeName.CHAR) {
+newLiteral = builder.makeLiteral(((NlsString) 
oplit.getValue()).getValue() + "%");
+input = operands.get(1);
+  }
+}
+else if (operands.get(1) instanceof RexLiteral) {
+  oplit = (RexLiteral) operands.get(1);
+  if(oplit.getTypeName() == SqlTypeName.CHAR) {
 
 Review comment:
   space 


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexSelector.java
 ##
 @@ -0,0 +1,766 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.cost.PluginCost;
+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 org.apache.drill.exec.planner.common.DrillScanRelBase;
+
+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;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+
+public class IndexSelector  {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(IndexSelector.class);
+  private static final double COVERING_TO_NONCOVERING_FACTOR = 100.0;
+  private RexNode indexCondition;   // filter condition on indexed columns
+  private RexNode otherRemainderCondition;  // remainder condition on all 
other columns
+  private double totalRows;
+  private Statistics stats; // a Statistics instance that will be used 
to get estimated rowcount for filter conditions
+  private IndexConditionInfo.Builder builder;
+  private List indexPropList;
+  private DrillScanRelBase primaryTableScan;
+  private IndexCallContext indexContext;
+  private RexBuilder rexBuilder;
+
+  public IndexSelector(RexNode indexCondition,
+  RexNode otherRemainderCondition,
+  IndexCallContext indexContext,
+  IndexCollection collection,
+  RexBuilder rexBuilder,
+  double totalRows) {
+this.indexCondition = indexCondition;
+this.otherRemainderCondition = otherRemainderCondition;
+this.indexContext = indexContext;
+this.totalRows = totalRows;
+this.stats = indexContext.getGroupScan().getStatistics();
+this.rexBuilder = rexBuilder;
+this.builder =
+IndexConditionInfo.newBuilder(indexCondition, collection, rexBuilder, 
indexContext.getScan());
+this.primaryTableScan = indexContext.getScan();
+this.indexPropList = Lists.newArrayList();
+  }
+
+  /**
+   * This constructor is to build selector for no index condition case (no 
filter)
+   * @param indexContext
+   */
+  public IndexSelector(IndexCallContext indexContext) {
+this.indexCondition = null;
+this.otherRemainderCondition = null;
+this.indexContext = indexContext;
+this.totalRows = Statistics.ROWCOUNT_UNKNOWN;
+this.stats = indexContext.getGroupScan().getStatistics();
+this.rexBuilder = indexContext.getScan().getCluster().getRexBuilder();
+this.builder = null;
+this.primaryTableScan = indexContext.getScan();
+this.indexPropList = Lists.newArrayList();
+  }
+
+  public void addIndex(IndexDescriptor indexDesc, boolean isCovering, int 
numProjectedFields) {
+IndexProperties indexProps = new DrillIndexProperties(indexDesc, 
isCovering, otherRemainderCondition, rexBuilder,
+numProjectedFields, totalRows, primaryTableScan);
+indexPropList.add(indexProps);
+  }
+
+  /**
+   * This method analyzes an index's columns and starting from the first 
column, checks
+ 

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 ##
 @@ -0,0 +1,872 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.common.OrderedRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+NONE,
+PARTIAL,
+FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of 
LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition 
on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, 
some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied 
as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker 
exprMarker, IndexDescriptor indexDesc) {
+Map mapRexExpr = 
exprMarker.getIndexableExpression();
+List infoCols = Lists.newArrayList();
+infoCols.addAll(mapRexExpr.values());
+if (indexDesc.allColumnsIndexed(infoCols)) {
+  return ConditionIndexed.FULL;
+} else if (indexDesc.someColumnsIndexed(infoCols)) {
+  return ConditionIndexed.PARTIAL;
+} else {
+  return ConditionIndexed.NONE;
+}
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan 
instance does not support secondary index, or
+   *this scan is already an index scan or Restricted Scan, do not apply 
index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+GroupScan groupScan = scanRel.getGroupScan();
+if (groupScan instanceof DbGroupScan) {
+  

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
 ##
 @@ -0,0 +1,250 @@
+/*
+ * 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.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import 
org.apache.drill.exec.planner.logical.partition.RewriteCombineBinaryOperators;
+import org.apache.calcite.rel.RelNode;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class IndexConditionInfo {
+  public final RexNode indexCondition;
+  public final RexNode remainderCondition;
+  public final boolean hasIndexCol;
+
+  public IndexConditionInfo(RexNode indexCondition, RexNode 
remainderCondition, boolean hasIndexCol) {
+this.indexCondition = indexCondition;
+this.remainderCondition = remainderCondition;
+this.hasIndexCol = hasIndexCol;
+  }
+
+  public static Builder newBuilder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan) {
+return new Builder(condition, indexes, builder, scan);
+  }
+
+  public static class Builder {
+final RexBuilder builder;
+final RelNode scan;
+final Iterable indexes;
+private RexNode condition;
+
+public Builder(RexNode condition,
+   Iterable indexes,
+   RexBuilder builder,
+   RelNode scan
+) {
 
 Review comment:
   formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
 ##
 @@ -0,0 +1,76 @@
+/*
+ * 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.physical.base;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.planner.index.Statistics;
+
+
+import java.util.List;
+
+/**
+ * An IndexGroupScan operator represents the scan associated with an Index.
+ */
+public interface IndexGroupScan extends GroupScan {
+
+  /**
+   * Get the column ordinal of the rowkey column from the output schema of the 
IndexGroupScan
+   * @return
+   */
+  @JsonIgnore
+  public int getRowKeyOrdinal();
+
+  /**
+   * Set the artificial row count after applying the {@link RexNode} condition
+   * Mainly used for debugging
+   * @param condition
+   * @param count
+   * @param capRowCount
+   */
+  @JsonIgnore
+  public void setRowCount(RexNode condition, double count, double capRowCount);
+
+  /**
+   * Get the row count after applying the {@link RexNode} condition
+   * @param condition, filter to apply
+   * @return row count post filtering
+   */
+  @JsonIgnore
+  public double getRowCount(RexNode condition, RelNode scanRel);
+
+  /**
+   * Set the statistics for {@link IndexGroupScan}
+   * @param statistics
+   */
+  @JsonIgnore
+  public void setStatistics(Statistics statistics);
+
+  @JsonIgnore
+  public void setColumns(List columns);
+
+  @JsonIgnore
+  public List getColumns();
+
+  @JsonIgnore
+  public void setParallelizationWidth(int width);
+
+}
 
 Review comment:
   new line


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class DrillIndexDefinition implements IndexDefinition {
+  /**
+   * The indexColumns is the list of column(s) on which this index is created. 
If there is more than 1 column,
+   * the order of the columns is important: index on {a, b} is not the same as 
index on {b, a}
+   * NOTE: the indexed column could be of type columnfamily.column
+   */
+  @JsonProperty
+  protected final List indexColumns;
+
+  /**
+   * nonIndexColumns: the list of columns that are included in the index as 
'covering'
+   * columns but are not themselves indexed.  These are useful for covering 
indexes where the
+   * query request can be satisfied directly by the index and avoid accessing 
the table altogether.
+   */
+  @JsonProperty
+  protected final List nonIndexColumns;
+
+  @JsonIgnore
+  protected final Set allIndexColumns;
+
+  @JsonProperty
+  protected final List rowKeyColumns;
+
+  @JsonProperty
+  protected final CollationContext indexCollationContext;
+
+  /**
+   * indexName: name of the index that should be unique within the scope of a 
table
+   */
+  @JsonProperty
+  protected final String indexName;
+
+  protected final String tableName;
+
+  @JsonProperty
+  protected final IndexDescriptor.IndexType indexType;
+
+  @JsonProperty
+  protected final NullDirection nullsDirection;
+
+  public DrillIndexDefinition(List indexCols,
+  CollationContext indexCollationContext,
+  List nonIndexCols,
+  List rowKeyColumns,
+  String indexName,
+  String tableName,
+  IndexType type,
+  NullDirection nullsDirection) {
+this.indexColumns = indexCols;
+this.nonIndexColumns = nonIndexCols;
+this.rowKeyColumns = rowKeyColumns;
+this.indexName = indexName;
+this.tableName = tableName;
+this.indexType = type;
+this.allIndexColumns = Sets.newHashSet(indexColumns);
+this.allIndexColumns.addAll(nonIndexColumns);
+this.indexCollationContext = indexCollationContext;
+this.nullsDirection = nullsDirection;
+
+  }
+
+  @Override
+  public int getIndexColumnOrdinal(LogicalExpression path) {
+int id = indexColumns.indexOf(path);
+return id;
+  }
+
+  @Override
+  public boolean isCoveringIndex(List columns) {
+return allIndexColumns.containsAll(columns);
+  }
+
+  @Override
+  public boolean allColumnsIndexed(Collection columns) {
+return columnsInIndexFields(columns, indexColumns);
+  }
+
+  @Override
+  public boolean someColumnsIndexed(Collection columns) {
+return someColumnsInIndexFields(columns, indexColumns);
+  }
+
+  public boolean pathExactIn(SchemaPath path, Collection 
exprs) {
+for (LogicalExpression expr : exprs) {
+  if (expr instanceof SchemaPath) {
+if (((SchemaPath) expr).toExpr().equals(path.toExpr())) {
+  return true;
+}
+  }
+}
+
+return false;
+  }
+
+  boolean castIsCompatible(CastExpression castExpr, 
Collection indexFields) {
+for(LogicalExpression indexExpr : 

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
 ##
 @@ -0,0 +1,284 @@
+/*
+ * 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.physical.impl.join;
+
+
+import java.util.List;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.SchemaChangeCallBack;
+import org.apache.drill.exec.vector.ValueVector;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+
+public class RowKeyJoinBatch extends AbstractRecordBatch 
implements RowKeyJoin {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RowKeyJoinBatch.class);
+
+  // primary table side record batch
+  private final RecordBatch left;
+
+  // index table side record batch
+  private final RecordBatch right;
+
+  private boolean hasRowKeyBatch;
+  private IterOutcome leftUpstream = IterOutcome.NONE;
+  private IterOutcome rightUpstream = IterOutcome.NONE;
+  private final List transfers = Lists.newArrayList();
+  private int recordCount = 0;
+  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
+  private RowKeyJoinState rkJoinState = RowKeyJoinState.INITIAL;
+
+  public RowKeyJoinBatch(RowKeyJoinPOP config, FragmentContext context, 
RecordBatch left, RecordBatch right)
+  throws OutOfMemoryException {
+super(config, context, true /* need to build schema */);
+this.left = left;
+this.right = right;
+this.hasRowKeyBatch = false;
+  }
+
+  @Override
+  public int getRecordCount() {
+if (state == BatchState.DONE) {
+  return 0;
+}
+return recordCount;
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+throw new UnsupportedOperationException("RowKeyJoinBatch does not support 
selection vector");
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+throw new UnsupportedOperationException("RowKeyJoinBatch does not support 
selection vector");
+  }
+
+  @Override
+  protected void buildSchema() throws SchemaChangeException {
+container.clear();
+
+rightUpstream = next(right);
+
+if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) 
{
+  state = BatchState.STOP;
+  return;
+}
+
+if (right.getRecordCount() > 0) {
+  // set the hasRowKeyBatch flag such that calling next() on the left input
+  // would see the correct status
+  hasRowKeyBatch = true;
+}
+
+leftUpstream = next(left);
+
+if (leftUpstream == IterOutcome.OUT_OF_MEMORY || rightUpstream == 
IterOutcome.OUT_OF_MEMORY) {
+  state = BatchState.OUT_OF_MEMORY;
+  return;
+}
+
+for(final VectorWrapper v : left) {
 
 Review comment:
   space


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
 ##
 @@ -0,0 +1,127 @@
+/*
+ * 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.db.MapRDB;
+import org.apache.drill.exec.util.GuavaPatcher;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+* Copy classes to a MapR cluster node, then run a command like this:
+* java -classpath 
/tmp/drill-cmd-1.9.0-SNAPSHOT.jar:/opt/mapr/drill/drill-1.9.0/jars/*:/opt/mapr/drill/drill-1.9.0/jars/3rdparty/*:/opt/mapr/drill/drill-1.9.0/jars/ext/*
+* org.apache.drill.hbase.index.TableIndexGen -host 
10.10.88.128 -port 5181 [-table pop3] [-size 100]
+*/
+
+class TestBigTable {
+
+  Admin admin;
+  boolean initialized = false;
+
+  LargeTableGen gen;
+
+  /*
+"hbase.zookeeper.quorum": "10.10.88.128",
+"hbase.zookeeper.property.clientPort": "5181"
+   */
+  void init(String host, String port) {
+try {
+  admin = MapRDB.newAdmin();
+  initialized = true;
+  gen = new LargeTableGen(admin);
+} catch (Exception e) {
+  System.out.println("Connection to HBase threw" + e.getMessage());
+}
+  }
+}
+
+
+public class TableIndexCmd {
+
+  public static Map parseParameter(String[] params) {
+HashMap retParams = new HashMap();
+for (int i=0; i

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

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

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
 ##
 @@ -0,0 +1,176 @@
+/*
+ * 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 static com.mapr.drill.maprdb.tests.MaprDBTestsSuite.INDEX_FLUSH_TIMEOUT;
+
+import java.io.InputStream;
+import java.io.StringBufferInputStream;
+
+import org.apache.hadoop.fs.Path;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
+
+import com.mapr.db.Admin;
+import com.mapr.db.Table;
+import com.mapr.db.TableDescriptor;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.impl.TableDescriptorImpl;
+import com.mapr.db.tests.utils.DBTests;
+import com.mapr.fs.utils.ssh.TestCluster;
+
+/**
+ * This class is to generate a MapR json table of this schema:
+ * {
+ *   "address" : {
+ *  "city":"wtj",
+ *  "state":"ho"
+ *   }
+ *   "contact" : {
+ *  "email":"vcfahj...@gmail.com",
+ *  "phone":"655583"
+ *   }
+ *   "id" : {
+ *  "ssn":"15461"
+ *   }
+ *   "name" : {
+ *  "fname":"VcFahj",
+ *  "lname":"RfM"
+ *   }
+ * }
+ *
+ */
+public class LargeTableGen extends LargeTableGenBase {
+
+  static final int SPLIT_SIZE = 5000;
+  private Admin admin;
+
+  public LargeTableGen(Admin dbadmin) {
+admin = dbadmin;
+  }
+
+  Table createOrGetTable(String tableName, int recordNum) {
+if (admin.tableExists(tableName)) {
+  return MapRDBImpl.getTable(tableName);
+  //admin.deleteTable(tableName);
+}
+else {
+  TableDescriptor desc = new TableDescriptorImpl(new Path(tableName));
+
+  int splits = (recordNum / SPLIT_SIZE) - (((recordNum % SPLIT_SIZE) > 1)? 
0 : 1);
+
+  String[] splitsStr = new String[splits];
+  StringBuilder strBuilder = new StringBuilder("Splits:");
+  for(int i=0; i

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

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

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
 ##
 @@ -0,0 +1,284 @@
+/*
+ * 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.physical.impl.join;
+
+
+import java.util.List;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.SchemaChangeCallBack;
+import org.apache.drill.exec.vector.ValueVector;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+
+public class RowKeyJoinBatch extends AbstractRecordBatch 
implements RowKeyJoin {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RowKeyJoinBatch.class);
+
+  // primary table side record batch
+  private final RecordBatch left;
+
+  // index table side record batch
+  private final RecordBatch right;
+
+  private boolean hasRowKeyBatch;
+  private IterOutcome leftUpstream = IterOutcome.NONE;
+  private IterOutcome rightUpstream = IterOutcome.NONE;
+  private final List transfers = Lists.newArrayList();
+  private int recordCount = 0;
+  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
+  private RowKeyJoinState rkJoinState = RowKeyJoinState.INITIAL;
+
+  public RowKeyJoinBatch(RowKeyJoinPOP config, FragmentContext context, 
RecordBatch left, RecordBatch right)
+  throws OutOfMemoryException {
+super(config, context, true /* need to build schema */);
+this.left = left;
+this.right = right;
+this.hasRowKeyBatch = false;
+  }
+
+  @Override
+  public int getRecordCount() {
+if (state == BatchState.DONE) {
+  return 0;
+}
+return recordCount;
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+throw new UnsupportedOperationException("RowKeyJoinBatch does not support 
selection vector");
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+throw new UnsupportedOperationException("RowKeyJoinBatch does not support 
selection vector");
+  }
+
+  @Override
+  protected void buildSchema() throws SchemaChangeException {
+container.clear();
+
+rightUpstream = next(right);
+
+if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) 
{
+  state = BatchState.STOP;
+  return;
+}
+
+if (right.getRecordCount() > 0) {
+  // set the hasRowKeyBatch flag such that calling next() on the left input
+  // would see the correct status
+  hasRowKeyBatch = true;
+}
+
+leftUpstream = next(left);
+
+if (leftUpstream == IterOutcome.OUT_OF_MEMORY || rightUpstream == 
IterOutcome.OUT_OF_MEMORY) {
+  state = BatchState.OUT_OF_MEMORY;
+  return;
+}
+
+for(final VectorWrapper v : left) {
+  final TransferPair pair = v.getValueVector().makeTransferPair(
+  container.addOrGet(v.getField(), callBack));
+  transfers.add(pair);
+}
+
+container.buildSchema(left.getSchema().getSelectionVectorMode());
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+if (state == BatchState.DONE) {
+  return IterOutcome.NONE;
+}
+try {
+  if (state == BatchState.FIRST && left.getRecordCount() > 0) {
+logger.debug("First batch, outputting the batch with {} records.", 
left.getRecordCount());
+// there is already a pending batch from left, output it
+outputCurrentLeftBatch();
+// 

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

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

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
 ##
 @@ -0,0 +1,127 @@
+/*
+ * 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.db.MapRDB;
+import org.apache.drill.exec.util.GuavaPatcher;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+* Copy classes to a MapR cluster node, then run a command like this:
+* java -classpath 
/tmp/drill-cmd-1.9.0-SNAPSHOT.jar:/opt/mapr/drill/drill-1.9.0/jars/*:/opt/mapr/drill/drill-1.9.0/jars/3rdparty/*:/opt/mapr/drill/drill-1.9.0/jars/ext/*
+* org.apache.drill.hbase.index.TableIndexGen -host 
10.10.88.128 -port 5181 [-table pop3] [-size 100]
+*/
+
+class TestBigTable {
+
+  Admin admin;
+  boolean initialized = false;
+
+  LargeTableGen gen;
+
+  /*
+"hbase.zookeeper.quorum": "10.10.88.128",
+"hbase.zookeeper.property.clientPort": "5181"
+   */
+  void init(String host, String port) {
+try {
+  admin = MapRDB.newAdmin();
+  initialized = true;
+  gen = new LargeTableGen(admin);
+} catch (Exception e) {
+  System.out.println("Connection to HBase threw" + e.getMessage());
+}
+  }
+}
+
+
+public class TableIndexCmd {
+
+  public static Map parseParameter(String[] params) {
+HashMap retParams = new HashMap();
+for (int i=0; i params = parseParameter(args);
+if(args.length >= 2) {
 
 Review comment:
   formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
 ##
 @@ -0,0 +1,186 @@
+/*
+ * 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 org.apache.commons.lang3.RandomStringUtils;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+public class LargeTableGenBase {
+
+  private boolean dict_ready = false;
+
+  protected List firstnames;
+  protected List lastnames;
+  protected List cities;
+  protected int[] randomized;
+
+  protected synchronized void  initDictionary() {
+initDictionaryWithRand();
+  }
+
+  protected void initDictionaryWithRand() {
+{
+  firstnames = new ArrayList<>();
+  lastnames = new ArrayList<>();
+  cities = new ArrayList<>();
+  List states = new ArrayList<>();
+
+  int fnNum = 2000; //2k
+  int lnNum = 20;//200k
+  int cityNum = 1;//10k
+  int stateNum = 50;
+  Random rand = new Random(2017);
+  int i;
+  try {
+Set strSet = new LinkedHashSet<>();
+while(strSet.size() < stateNum) {
+  strSet.add(RandomStringUtils.random(2, 0, 0, true, false, null, 
rand));
+}
+states.addAll(strSet);
+
+strSet = new LinkedHashSet<>();
+while(strSet.size() < cityNum) {
+  int len = 3 + strSet.size() % 6;
+  strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, 
rand));
+}
+
+Iterator it = strSet.iterator();
+for(i=0; i

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
 ##
 @@ -0,0 +1,65 @@
+/*
+ * 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.udf.mapr.db;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import io.netty.buffer.DrillBuf;
+
+@FunctionTemplate(name = "maprdb_decode_fieldpath", scope = 
FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DecodeFieldPath implements DrillSimpleFunc {
+  @Param  VarCharHolder input;
+  @Output VarCharHolder   out;
+
+  @Inject DrillBuf buffer;
+
+  @Override
+  public void setup() {
+  }
+
+  @Override
+  public void eval() {
+String[] encodedPaths = 
org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.
+toStringFromUTF8(input.start, input.end, input.buffer).split(",");
+String[] decodedPaths = 
org.apache.drill.exec.util.EncodedSchemaPathSet.decode(encodedPaths);
+java.util.Arrays.sort(decodedPaths);
+
+StringBuilder sb = new StringBuilder();
+for(String decodedPath : decodedPaths) {
 
 Review comment:
   sapce


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
 ##
 @@ -0,0 +1,214 @@
+/*
+ * 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.json;
+
+import org.apache.commons.codec.binary.Base64;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import org.ojai.Value;
+import org.ojai.store.QueryCondition;
+
+import com.google.common.collect.ImmutableMap;
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.MapRDBImpl;
+
+import java.nio.ByteBuffer;
+
+class OjaiFunctionsProcessor extends AbstractExprVisitor {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(OjaiFunctionsProcessor.class);
+  private QueryCondition queryCond;
+
+  private OjaiFunctionsProcessor() {
+  }
+
+  private static String getStackTrace() {
+final Throwable throwable = new Throwable();
+final StackTraceElement[] ste = throwable.getStackTrace();
+final StringBuilder sb = new StringBuilder();
+for(int i = 1; i < ste.length; ++i) {
 
 Review comment:
   space


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
 ##
 @@ -0,0 +1,184 @@
+/*
+ * 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.json;
+
+import java.util.List;
+import java.util.NavigableMap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.index.MapRDBStatistics;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.Statistics;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.TabletFragmentInfo;
+
+/**
+ * A RestrictedJsonTableGroupScan encapsulates (along with a subscan) the 
functionality
+ * for doing restricted (i.e skip) scan rather than sequential scan.  The 
skipping is based
+ * on a supplied set of row keys (primary keys) from a join operator.
+ */
+@JsonTypeName("restricted-json-scan")
+public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
+
+  @JsonCreator
+  public RestrictedJsonTableGroupScan(@JsonProperty("userName") String 
userName,
+@JsonProperty("storage") FileSystemPlugin 
storagePlugin,
+@JsonProperty("format") MapRDBFormatPlugin 
formatPlugin,
+@JsonProperty("scanSpec") JsonScanSpec scanSpec, 
/* scan spec of the original table */
+@JsonProperty("columns") List columns,
+@JsonProperty("")MapRDBStatistics statistics) {
+super(userName, storagePlugin, formatPlugin, scanSpec, columns, 
statistics);
+  }
+
+  // TODO:  this method needs to be fully implemented
+  protected RestrictedMapRDBSubScanSpec getSubScanSpec(TabletFragmentInfo tfi) 
{
+JsonScanSpec spec = scanSpec;
+RestrictedMapRDBSubScanSpec subScanSpec =
+new RestrictedMapRDBSubScanSpec(
+spec.getTableName(),
+getRegionsToScan().get(tfi), spec.getSerializedFilter(), 
getUserName());
+return subScanSpec;
+  }
+
+  protected NavigableMap getRegionsToScan() {
+return getRegionsToScan(formatPlugin.getRestrictedScanRangeSizeMB());
+  }
+
+  @Override
+  public MapRDBSubScan getSpecificScan(int minorFragmentId) {
+assert minorFragmentId < endpointFragmentMapping.size() : String.format(
+"Mappings length [%d] should be greater than minor fragment id [%d] 
but it isn't.", endpointFragmentMapping.size(),
+minorFragmentId);
+RestrictedMapRDBSubScan subscan =
+new RestrictedMapRDBSubScan(getUserName(), formatPlugin,
+getEndPointFragmentMapping(minorFragmentId), columns, 
maxRecordsToRead, TABLE_JSON);
+
+return subscan;
+  }
+
+  private List getEndPointFragmentMapping(int 
minorFragmentId) {
+List restrictedSubScanSpecList = 
Lists.newArrayList();
+List subScanSpecList = 
endpointFragmentMapping.get(minorFragmentId);
+for(MapRDBSubScanSpec s : subScanSpecList) {
 
 Review comment:
   space


This is an automated message from the Apache Git 

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
 ##
 @@ -0,0 +1,184 @@
+/*
+ * 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.json;
+
+import java.util.List;
+import java.util.NavigableMap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.index.MapRDBStatistics;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.Statistics;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.TabletFragmentInfo;
+
+/**
+ * A RestrictedJsonTableGroupScan encapsulates (along with a subscan) the 
functionality
+ * for doing restricted (i.e skip) scan rather than sequential scan.  The 
skipping is based
+ * on a supplied set of row keys (primary keys) from a join operator.
+ */
+@JsonTypeName("restricted-json-scan")
+public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
+
+  @JsonCreator
+  public RestrictedJsonTableGroupScan(@JsonProperty("userName") String 
userName,
+@JsonProperty("storage") FileSystemPlugin 
storagePlugin,
+@JsonProperty("format") MapRDBFormatPlugin 
formatPlugin,
+@JsonProperty("scanSpec") JsonScanSpec scanSpec, 
/* scan spec of the original table */
+@JsonProperty("columns") List columns,
+@JsonProperty("")MapRDBStatistics statistics) {
 
 Review comment:
   space


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.json;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.exec.planner.physical.AbstractRangePartitionFunction;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.vector.ValueVector;
+import org.ojai.store.QueryCondition;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.mapr.db.Table;
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.impl.ConditionNode.RowkeyRange;
+import com.mapr.db.scan.ScanRange;
+import com.mapr.fs.jni.MapRConstants;
+import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+
+@JsonTypeName("jsontable-range-partition-function")
+public class JsonTableRangePartitionFunction extends 
AbstractRangePartitionFunction {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(JsonTableRangePartitionFunction.class);
+
+  @JsonProperty("refList")
+  protected List refList;
+
+  @JsonProperty("tableName")
+  protected String tableName;
+
+  @JsonIgnore
+  protected String userName;
+
+  @JsonIgnore
+  protected ValueVector partitionKeyVector = null;
+
+  // List of start keys of the scan ranges for the table.
+  @JsonProperty
+  protected List startKeys = null;
+
+  // List of stop keys of the scan ranges for the table.
+  @JsonProperty
+  protected List stopKeys = null;
+
+  @JsonCreator
+  public JsonTableRangePartitionFunction(
+  @JsonProperty("refList") List refList,
+  @JsonProperty("tableName") String tableName,
+  @JsonProperty("startKeys") List startKeys,
+  @JsonProperty("stopKeys") List stopKeys) {
+this.refList = refList;
+this.tableName = tableName;
+this.startKeys = startKeys;
+this.stopKeys = stopKeys;
+  }
+
+  public JsonTableRangePartitionFunction(List refList,
+  String tableName, String userName, MapRDBFormatPlugin formatPlugin) {
+this.refList = refList;
+this.tableName = tableName;
+this.userName = userName;
+initialize(formatPlugin);
+  }
+
+  @JsonProperty("refList")
+  @Override
+  public List getPartitionRefList() {
+return refList;
+  }
+
+  @Override
+  public void setup(List> partitionKeys) {
+if (partitionKeys.size() != 1) {
+  throw new UnsupportedOperationException(
+  "Range partitioning function supports exactly one partition column; 
encountered " + partitionKeys.size());
+}
+
+VectorWrapper v = partitionKeys.get(0);
+
+partitionKeyVector = v.getValueVector();
+
+Preconditions.checkArgument(partitionKeyVector != null, "Found null 
partitionKeVector.") ;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+if (this == obj) {
+  return true;
+}
+if (obj instanceof JsonTableRangePartitionFunction) {
+  JsonTableRangePartitionFunction rpf = (JsonTableRangePartitionFunction) 
obj;
+  List thisPartRefList = this.getPartitionRefList();
+  List otherPartRefList = rpf.getPartitionRefList();
+  if (thisPartRefList.size() != otherPartRefList.size()) {
+return false;
+  }
+  for (int refIdx=0; refIdx= 0 ||
 
 Review comment:
   formatting


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



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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
 ##
 @@ -214,11 +445,323 @@ public boolean canPushdownProjects(List 
columns) {
 
   @Override
   public String toString() {
-return "JsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns 
+ "]";
+return "JsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns
++ (maxRecordsToRead>0? ", limit=" + maxRecordsToRead : "")
 
 Review comment:
   formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.json;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.exec.planner.physical.AbstractRangePartitionFunction;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.vector.ValueVector;
+import org.ojai.store.QueryCondition;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.mapr.db.Table;
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.impl.ConditionNode.RowkeyRange;
+import com.mapr.db.scan.ScanRange;
+import com.mapr.fs.jni.MapRConstants;
+import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+
+@JsonTypeName("jsontable-range-partition-function")
+public class JsonTableRangePartitionFunction extends 
AbstractRangePartitionFunction {
+
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(JsonTableRangePartitionFunction.class);
+
+  @JsonProperty("refList")
+  protected List refList;
+
+  @JsonProperty("tableName")
+  protected String tableName;
+
+  @JsonIgnore
+  protected String userName;
+
+  @JsonIgnore
+  protected ValueVector partitionKeyVector = null;
+
+  // List of start keys of the scan ranges for the table.
+  @JsonProperty
+  protected List startKeys = null;
+
+  // List of stop keys of the scan ranges for the table.
+  @JsonProperty
+  protected List stopKeys = null;
+
+  @JsonCreator
+  public JsonTableRangePartitionFunction(
+  @JsonProperty("refList") List refList,
+  @JsonProperty("tableName") String tableName,
+  @JsonProperty("startKeys") List startKeys,
+  @JsonProperty("stopKeys") List stopKeys) {
+this.refList = refList;
+this.tableName = tableName;
+this.startKeys = startKeys;
+this.stopKeys = stopKeys;
+  }
+
+  public JsonTableRangePartitionFunction(List refList,
+  String tableName, String userName, MapRDBFormatPlugin formatPlugin) {
+this.refList = refList;
+this.tableName = tableName;
+this.userName = userName;
+initialize(formatPlugin);
+  }
+
+  @JsonProperty("refList")
+  @Override
+  public List getPartitionRefList() {
+return refList;
+  }
+
+  @Override
+  public void setup(List> partitionKeys) {
+if (partitionKeys.size() != 1) {
+  throw new UnsupportedOperationException(
+  "Range partitioning function supports exactly one partition column; 
encountered " + partitionKeys.size());
+}
+
+VectorWrapper v = partitionKeys.get(0);
+
+partitionKeyVector = v.getValueVector();
+
+Preconditions.checkArgument(partitionKeyVector != null, "Found null 
partitionKeVector.") ;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+if (this == obj) {
+  return true;
+}
+if (obj instanceof JsonTableRangePartitionFunction) {
+  JsonTableRangePartitionFunction rpf = (JsonTableRangePartitionFunction) 
obj;
+  List thisPartRefList = this.getPartitionRefList();
+  List otherPartRefList = rpf.getPartitionRefList();
+  if (thisPartRefList.size() != otherPartRefList.size()) {
+return false;
+  }
+  for (int refIdx=0; refIdx

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
 ##
 @@ -98,91 +114,181 @@
   private final boolean disableCountOptimization;
   private final boolean nonExistentColumnsProjection;
 
-  public MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec,
-  MapRDBFormatPluginConfig formatPluginConfig,
-  List projectedColumns, FragmentContext context) {
+  protected final MapRDBSubScanSpec subScanSpec;
+  protected final MapRDBFormatPlugin formatPlugin;
+
+  protected OjaiValueWriter valueWriter;
+  protected DocumentReaderVectorWriter documentWriter;
+  protected int maxRecordsToRead = -1;
+
+  public MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec, 
MapRDBFormatPlugin formatPlugin,
+List projectedColumns, 
FragmentContext context, int maxRecords) {
+this(subScanSpec, formatPlugin, projectedColumns, context);
+this.maxRecordsToRead = maxRecords;
+  }
+
+  protected MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec, 
MapRDBFormatPlugin formatPlugin,
+List projectedColumns, 
FragmentContext context) {
 buffer = context.getManagedBuffer();
-projectedFields = null;
-tableName = Preconditions.checkNotNull(subScanSpec, "MapRDB reader needs a 
sub-scan spec").getTableName();
-documentReaderIterators = null;
-includeId = false;
-idOnly= false;
+final Path tablePath = new Path(Preconditions.checkNotNull(subScanSpec,
+  "MapRDB reader needs a sub-scan spec").getTableName());
+this.subScanSpec = subScanSpec;
+this.formatPlugin = formatPlugin;
+final IndexDesc indexDesc = subScanSpec.getIndexDesc();
 byte[] serializedFilter = subScanSpec.getSerializedFilter();
 condition = null;
 
 if (serializedFilter != null) {
   condition = 
com.mapr.db.impl.ConditionImpl.parseFrom(ByteBufs.wrap(serializedFilter));
 }
 
-disableCountOptimization = formatPluginConfig.disableCountOptimization();
+disableCountOptimization = 
formatPlugin.getConfig().disableCountOptimization();
+// Below call will set the scannedFields and includeId correctly
 setColumns(projectedColumns);
-unionEnabled = 
context.getOptions().getBoolean(ExecConstants.ENABLE_UNION_TYPE_KEY);
-readNumbersAsDouble = formatPluginConfig.isReadAllNumbersAsDouble();
-allTextMode = formatPluginConfig.isAllTextMode();
-ignoreSchemaChange = formatPluginConfig.isIgnoreSchemaChange();
-disablePushdown = !formatPluginConfig.isEnablePushdown();
-nonExistentColumnsProjection = 
formatPluginConfig.isNonExistentFieldSupport();
+unionEnabled = 
context.getOptions().getOption(ExecConstants.ENABLE_UNION_TYPE);
+readNumbersAsDouble = formatPlugin.getConfig().isReadAllNumbersAsDouble();
+allTextMode = formatPlugin.getConfig().isAllTextMode();
+ignoreSchemaChange = formatPlugin.getConfig().isIgnoreSchemaChange();
+disablePushdown = !formatPlugin.getConfig().isEnablePushdown();
+nonExistentColumnsProjection = 
formatPlugin.getConfig().isNonExistentFieldSupport();
+
+// Do not use cached table handle for two reasons.
+// cached table handles default timeout is 60 min after which those 
handles will become stale.
+// Since execution can run for longer than 60 min, we want to get a new 
table handle and use it
+// instead of the one from cache.
+// Since we are setting some table options, we do not want to use shared 
handles.
+//
+// Call it here instead of setup since this will make sure it's called 
under correct UGI block when impersonation
+// is enabled and table is used with and without views.
+table = (indexDesc == null ? MapRDBImpl.getTable(tablePath) : 
MapRDBImpl.getIndexTable(indexDesc));
+
+if (condition != null) {
+  logger.debug("Created record reader with query condition {}", 
condition.toString());
+} else {
+  logger.debug("Created record reader with query condition NULL");
+}
   }
 
   @Override
   protected Collection transformColumns(Collection 
columns) {
 Set transformed = Sets.newLinkedHashSet();
+Set encodedSchemaPathSet = Sets.newLinkedHashSet();
+
 if (disablePushdown) {
   transformed.add(SchemaPath.STAR_COLUMN);
   includeId = true;
-  return transformed;
-}
+} else {
+  if (isStarQuery()) {
+transformed.add(SchemaPath.STAR_COLUMN);
+includeId = true;
+if (isSkipQuery() && !disableCountOptimization) {
+  // `SELECT COUNT(*)` query
+  idOnly = true;
+  scannedFields = ID_ONLY_PROJECTION;
+}
+  } else {
+Set scannedFieldsSet = Sets.newTreeSet();
+Set projectedFieldsSet = null;
 
-if (isStarQuery()) {
-  

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
 ##
 @@ -179,16 +295,126 @@ public MapRDBSubScan getSpecificScan(int 
minorFragmentId) {
 assert minorFragmentId < endpointFragmentMapping.size() : String.format(
 "Mappings length [%d] should be greater than minor fragment id [%d] 
but it isn't.", endpointFragmentMapping.size(),
 minorFragmentId);
-return new MapRDBSubScan(getUserName(), formatPlugin, 
endpointFragmentMapping.get(minorFragmentId), columns, TABLE_JSON);
+return new MapRDBSubScan(getUserName(), formatPlugin, 
endpointFragmentMapping.get(minorFragmentId), columns, maxRecordsToRead, 
TABLE_JSON);
   }
 
   @Override
   public ScanStats getScanStats() {
-//TODO: look at stats for this.
-long rowCount = (long) ((scanSpec.getSerializedFilter() != null ? .5 : 1) 
* totalRowCount);
-int avgColumnSize = 10;
-int numColumns = (columns == null || columns.isEmpty()) ? 100 : 
columns.size();
-return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, 
avgColumnSize * numColumns * rowCount);
+if (isIndexScan()) {
+  return indexScanStats();
+}
+return fullTableScanStats();
+  }
+
+  private ScanStats fullTableScanStats() {
+PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
+final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
+final int numColumns = (columns == null || columns.isEmpty()) ? STAR_COLS 
: columns.size();
+// index will be NULL for FTS
+double rowCount = stats.getRowCount(scanSpec.getCondition(), null);
+// rowcount based on _id predicate. If NO _id predicate present in 
condition, then the
+// rowcount should be same as totalRowCount. Equality b/w the two 
rowcounts should not be
+// construed as NO _id predicate since stats are approximate.
+double leadingRowCount = stats.getLeadingRowCount(scanSpec.getCondition(), 
null);
+double avgRowSize = stats.getAvgRowSize(null, true);
+double totalRowCount = stats.getRowCount(null, null);
+logger.debug("GroupScan {} with stats {}: rowCount={}, condition={}, 
totalRowCount={}, fullTableRowCount={}",
+System.identityHashCode(this), System.identityHashCode(stats), 
rowCount,
+scanSpec.getCondition()==null?"null":scanSpec.getCondition(),
 
 Review comment:
   formatting


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
 ##
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.expr.CloneVisitor;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.IndexProperties;
+import org.apache.drill.exec.store.mapr.PluginConstants;
+import org.apache.drill.exec.util.EncodedSchemaPathSet;
+import org.apache.drill.common.expression.LogicalExpression;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+public class MapRDBIndexDescriptor extends DrillIndexDescriptor {
+
+  protected final Object desc;
+  protected final Set allFields;
+  protected final Set indexedFields;
+  protected MapRDBFunctionalIndexInfo functionalInfo;
+  protected PluginCost pluginCost;
+
+  public MapRDBIndexDescriptor(List indexCols,
+   CollationContext indexCollationContext,
+   List nonIndexCols,
+   List rowKeyColumns,
+   String indexName,
+   String tableName,
+   IndexType type,
+   Object desc,
+   DbGroupScan scan,
+   NullDirection nullsDirection) {
+super(indexCols, indexCollationContext, nonIndexCols, rowKeyColumns, 
indexName, tableName, type, nullsDirection);
+this.desc = desc;
+this.indexedFields = ImmutableSet.copyOf(indexColumns);
+this.allFields = new ImmutableSet.Builder()
+.add(PluginConstants.DOCUMENT_SCHEMA_PATH)
+.addAll(indexColumns)
+.addAll(nonIndexColumns)
+.build();
+this.pluginCost = scan.getPluginCostModel();
+  }
+
+  public Object getOriginalDesc(){
+return desc;
+  }
+
+  @Override
+  public boolean isCoveringIndex(List expressions) {
+List decodedCols = new 
DecodePathinExpr().parseExpressions(expressions);
+return columnsInIndexFields(decodedCols, allFields);
+  }
+
+  @Override
+  public boolean allColumnsIndexed(Collection expressions) {
+List decodedCols = new 
DecodePathinExpr().parseExpressions(expressions);
+return columnsInIndexFields(decodedCols, indexedFields);
+  }
+
+  @Override
+  public boolean someColumnsIndexed(Collection columns) {
+return columnsIndexed(columns, false);
+  }
+
+  private boolean columnsIndexed(Collection expressions, 
boolean allColsIndexed) {
+List decodedCols = new 
DecodePathinExpr().parseExpressions(expressions);
+if (allColsIndexed) {
+  return columnsInIndexFields(decodedCols, indexedFields);
+} else {
+  return someColumnsInIndexFields(decodedCols, indexedFields);
+}
+  }
+
+  public FunctionalIndexInfo getFunctionalInfo() {
+if (this.functionalInfo == null) {
+  this.functionalInfo = new MapRDBFunctionalIndexInfo(this);
+}
+return this.functionalInfo;
+  }
+
+  /**
+   * Search through a LogicalExpression, finding all referenced schema paths
+   * and replace them with decoded paths.
+   * If one encoded path could be decoded to multiple paths, add these decoded 
paths to
+   * the end of returned list of expressions from parseExpressions.
+   */
+  private 

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,163 @@
+/*
+ * 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();
+exprToConvert = Maps.newHashMap();
+pathsInExpr = Maps.newHashMap();
+//keep the order of new paths, it may be related to the naming policy
+newPathsForIndexedFunction = Sets.newLinkedHashSet();
+allPathsInFunction = Sets.newHashSet();
+init();
+  }
+
+  private void init() {
+int count = 0;
+for(LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
+  if( !(indexedExpr instanceof SchemaPath) ) {
 
 Review comment:
   remove spaces please


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,163 @@
+/*
+ * 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();
+exprToConvert = Maps.newHashMap();
+pathsInExpr = Maps.newHashMap();
+//keep the order of new paths, it may be related to the naming policy
+newPathsForIndexedFunction = Sets.newLinkedHashSet();
+allPathsInFunction = Sets.newHashSet();
+init();
+  }
+
+  private void init() {
+int count = 0;
+for(LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
+  if( !(indexedExpr instanceof SchemaPath) ) {
+hasFunctionalField = true;
+SchemaPath functionalFieldPath = SchemaPath.getSimplePath("$"+count);
+newPathsForIndexedFunction.add(functionalFieldPath);
+
+//now we handle only cast expression
 
 Review comment:
   Please add white spaces in the comments


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] vdiravka commented on a change in pull request #1466: DRILL-6381: Add support for index based planning and execution

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 ##
 @@ -0,0 +1,163 @@
+/*
+ * 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();
+exprToConvert = Maps.newHashMap();
+pathsInExpr = Maps.newHashMap();
+//keep the order of new paths, it may be related to the naming policy
+newPathsForIndexedFunction = Sets.newLinkedHashSet();
+allPathsInFunction = Sets.newHashSet();
+init();
+  }
+
+  private void init() {
+int count = 0;
+for(LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
+  if( !(indexedExpr instanceof SchemaPath) ) {
+hasFunctionalField = true;
+SchemaPath functionalFieldPath = SchemaPath.getSimplePath("$"+count);
+newPathsForIndexedFunction.add(functionalFieldPath);
+
+//now we handle only cast expression
+if(indexedExpr instanceof CastExpression) {
+  //We handle only CAST directly on SchemaPath for now.
+  SchemaPath pathBeingCasted = (SchemaPath)((CastExpression) 
indexedExpr).getInput();
+  addTargetPathForOriginalPath(pathBeingCasted, functionalFieldPath);
+  addPathInExpr(indexedExpr, pathBeingCasted);
+  exprToConvert.put(indexedExpr, functionalFieldPath);
+  allPathsInFunction.add(pathBeingCasted);
+}
+
+count++;
+  }
+}
+  }
+
+  private void addPathInExpr(LogicalExpression expr, SchemaPath path) {
+if (!pathsInExpr.containsKey(expr)) {
+  Set newSet = Sets.newHashSet();
+  newSet.add(path);
+  pathsInExpr.put(expr, newSet);
+}
+else {
+  pathsInExpr.get(expr).add(path);
+}
+  }
+
+  private void addTargetPathForOriginalPath(SchemaPath origPath, SchemaPath 
newPath) {
+if (!columnToConvert.containsKey(origPath)) {
+  Set newSet = Sets.newHashSet();
+  newSet.add(newPath);
+  columnToConvert.put(origPath, newSet);
+}
+else {
+  columnToConvert.get(origPath).add(newPath);
+}
+  }
+
+
+  public boolean hasFunctional() {
+return hasFunctionalField;
+  }
+
+  public IndexDescriptor getIndexDesc() {
+return indexDesc;
+  }
+
+  /**
+   * getNewPath: for an original path, return new rename '$N' path, notice 
there could be multiple renamed paths
+   * if the there are multiple functional indexes refer original path.
+   * @param path
+   * @return
+   */
+  public SchemaPath getNewPath(SchemaPath path) {
+if(columnToConvert.containsKey(path)) {
 
 Review comment:
   add space


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific 

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

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

 ##
 File path: 
contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
 ##
 @@ -19,32 +19,39 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.mapr.db.index.IndexDesc;
 import com.mapr.fs.jni.MapRConstants;
 import com.mapr.org.apache.hadoop.hbase.util.Bytes;
 
-public class MapRDBSubScanSpec {
+public class MapRDBSubScanSpec implements Comparable{
 
 Review comment:
   space


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


With regards,
Apache Git Services


[jira] [Created] (DRILL-6791) Merge scan projection framework into master

2018-10-11 Thread Paul Rogers (JIRA)
Paul Rogers created DRILL-6791:
--

 Summary: Merge scan projection framework into master
 Key: DRILL-6791
 URL: https://issues.apache.org/jira/browse/DRILL-6791
 Project: Apache Drill
  Issue Type: Improvement
Affects Versions: 1.15.0
Reporter: Paul Rogers
Assignee: Paul Rogers


Merge the next set of "result set loader" code into master via a PR. This one 
covers the "schema projection" mechanism which:

* Handles none (SELECT COUNT(*)), some (SELECT a, b, x) and all (SELECT *) 
projection.
* Handles null columns (for projection a column "x" that does not exist in the 
base table.)
* Handle file metadata (AKA implicit columns)
* Handle schema persistence: the need to reuse the same vectors across 
different scanners
* Provides a framework for consuming externally-supplied metadata
* Baring "real" metadata, obtains metadata hints from previous batches and from 
the projection list (a.b implies that "a" is a map, c[0] implies that "c" is an 
array, etc.)




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


Re: Unsupported type LIST when CTAS arrayOfArray (JSON or Msgpack) into Parquet

2018-10-11 Thread Paul Rogers
I don't believe Parquet supports 2D arrays, does it?

Thanks,
- Paul

 

On Thursday, October 11, 2018, 7:52:38 PM PDT, Jean-Claude Cote 
 wrote:  
 
 I'm trying to write the following JSON file into a parquet file. However my
CTAS query returns an error Unsupported type LIST. Any ideas why, I'm
pretty sure parquet support array of array.

Thanks
jc

cat /tmp/json1/0_0_0.json
{
  "arrayOfArray" : [ [ 1, 1, 1 ], [ 1, 1, 1, 1, 1 ] ]
}

0: jdbc:drill:zk=local> create table dfs.tmp.`par3` as (select arrayOfArray
from dfs.tmp.`json1`);
Error: SYSTEM ERROR: UnsupportedOperationException: Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010] (state=,code=0)
java.sql.SQLException: SYSTEM ERROR: UnsupportedOperationException:
Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010]
        at
org.apache.drill.jdbc.impl.DrillCursor.nextRowInternally(DrillCursor.java:528)
        at
org.apache.drill.jdbc.impl.DrillCursor.loadInitialSchema(DrillCursor.java:600)
        at
org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:1288)
        at
org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:61)
        at
org.apache.calcite.avatica.AvaticaConnection$1.execute(AvaticaConnection.java:667)
        at
org.apache.drill.jdbc.impl.DrillMetaImpl.prepareAndExecute(DrillMetaImpl.java:1109)
        at
org.apache.drill.jdbc.impl.DrillMetaImpl.prepareAndExecute(DrillMetaImpl.java:1120)
        at
org.apache.calcite.avatica.AvaticaConnection.prepareAndExecuteInternal(AvaticaConnection.java:675)
        at
org.apache.drill.jdbc.impl.DrillConnectionImpl.prepareAndExecuteInternal(DrillConnectionImpl.java:196)
        at
org.apache.calcite.avatica.AvaticaStatement.executeInternal(AvaticaStatement.java:156)
        at
org.apache.calcite.avatica.AvaticaStatement.execute(AvaticaStatement.java:217)
        at sqlline.Commands.execute(Commands.java:814)
        at sqlline.Commands.sql(Commands.java:754)
        at sqlline.SqlLine.dispatch(SqlLine.java:646)
        at sqlline.SqlLine.begin(SqlLine.java:510)
        at sqlline.SqlLine.start(SqlLine.java:233)
        at sqlline.SqlLine.main(SqlLine.java:175)
Caused by: org.apache.drill.common.exceptions.UserRemoteException: SYSTEM
ERROR: UnsupportedOperationException: Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010]
        at
org.apache.drill.exec.rpc.user.QueryResultHandler.resultArrived(QueryResultHandler.java:123)
        at
org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:422)
        at
org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:96)
        at
org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:273)
        at
org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:243)
        at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
        at
io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:287)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
        at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
        at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:312)
        at
io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:286)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
        at
io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
        at

Unsupported type LIST when CTAS arrayOfArray (JSON or Msgpack) into Parquet

2018-10-11 Thread Jean-Claude Cote
I'm trying to write the following JSON file into a parquet file. However my
CTAS query returns an error Unsupported type LIST. Any ideas why, I'm
pretty sure parquet support array of array.

Thanks
jc

cat /tmp/json1/0_0_0.json
{
  "arrayOfArray" : [ [ 1, 1, 1 ], [ 1, 1, 1, 1, 1 ] ]
}

0: jdbc:drill:zk=local> create table dfs.tmp.`par3` as (select arrayOfArray
from dfs.tmp.`json1`);
Error: SYSTEM ERROR: UnsupportedOperationException: Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010] (state=,code=0)
java.sql.SQLException: SYSTEM ERROR: UnsupportedOperationException:
Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010]
at
org.apache.drill.jdbc.impl.DrillCursor.nextRowInternally(DrillCursor.java:528)
at
org.apache.drill.jdbc.impl.DrillCursor.loadInitialSchema(DrillCursor.java:600)
at
org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:1288)
at
org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:61)
at
org.apache.calcite.avatica.AvaticaConnection$1.execute(AvaticaConnection.java:667)
at
org.apache.drill.jdbc.impl.DrillMetaImpl.prepareAndExecute(DrillMetaImpl.java:1109)
at
org.apache.drill.jdbc.impl.DrillMetaImpl.prepareAndExecute(DrillMetaImpl.java:1120)
at
org.apache.calcite.avatica.AvaticaConnection.prepareAndExecuteInternal(AvaticaConnection.java:675)
at
org.apache.drill.jdbc.impl.DrillConnectionImpl.prepareAndExecuteInternal(DrillConnectionImpl.java:196)
at
org.apache.calcite.avatica.AvaticaStatement.executeInternal(AvaticaStatement.java:156)
at
org.apache.calcite.avatica.AvaticaStatement.execute(AvaticaStatement.java:217)
at sqlline.Commands.execute(Commands.java:814)
at sqlline.Commands.sql(Commands.java:754)
at sqlline.SqlLine.dispatch(SqlLine.java:646)
at sqlline.SqlLine.begin(SqlLine.java:510)
at sqlline.SqlLine.start(SqlLine.java:233)
at sqlline.SqlLine.main(SqlLine.java:175)
Caused by: org.apache.drill.common.exceptions.UserRemoteException: SYSTEM
ERROR: UnsupportedOperationException: Unsupported type LIST

Fragment 0:0

[Error Id: c4732fe5-b74a-4e55-a39d-f7338b1b9314 on
DESKTOP-1JUH8A1.localdomain:31010]
at
org.apache.drill.exec.rpc.user.QueryResultHandler.resultArrived(QueryResultHandler.java:123)
at
org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:422)
at
org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:96)
at
org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:273)
at
org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:243)
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
at
io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:287)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:312)
at
io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:286)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
at
io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
at

Re: msgpack format reader with schema learning feature

2018-10-11 Thread Jean-Claude Cote
I'm pretty sure I can have the type of the list. Here's an example of the
schema I use. The root map correspond to the row. Notice how the
arrayOfArray says it is a LIST:REPEATED. But if you drill down and ask for
the first child of that MaterializedField you get the inner array named
$data$ and it is of type BIGINT:REPEATED. So I know it's another array.
Then I drill down some more and I get to BIGINT:REQUIRED. That's the type
of the elements.

[`` (MAP:REQUIRED), children=(

[`arrayOfArray` (LIST:REPEATED), children=([`$data$` (BIGINT:REPEATED),
children=([`$data$` (BIGINT(0, 0):REQUIRED)])])],

 [`arrayOfMap` (MAP:REPEATED), children=([`raw` (VARCHAR:OPTIONAL)], [`bin`
(VARBINARY:OPTIONAL)], [`int` (BIGINT:OPTIONAL)], [`big`
(BIGINT:OPTIONAL)], [`byt` (BIGINT:OPTIONAL)], [`lon` (BIGINT:OPTIONAL)],
[`sho` (BIGINT:OPTIONAL)], [`dou` (FLOAT8:OPTIONAL)], [`flo`
(FLOAT8:OPTIONAL)], [`str` (VARCHAR:OPTIONAL)])],

 [`mapOfMap` (MAP:REQUIRED), children=([`aMap` (MAP:REQUIRED),
children=([`raw` (VARCHAR:OPTIONAL)], [`bin` (VARBINARY:OPTIONAL)], [`int`
(BIGINT:OPTIONAL)], [`big` (BIGINT:OPTIONAL)], [`byt` (BIGINT:OPTIONAL)],
[`lon` (BIGINT:OPTIONAL)], [`sho` (BIGINT:OPTIONAL)], [`dou`
(FLOAT8:OPTIONAL)], [`flo` (FLOAT8:OPTIONAL)], [`str`
(VARCHAR:OPTIONAL)])])],

[`mapWithArray` (MAP:REQUIRED), children=([`anArray` (VARCHAR:REPEATED),
children=([`$data$` (VARCHAR(0, 0):REQUIRED)])], [`aString`
(VARCHAR:OPTIONAL)])], [`

The problem I had with the TupleSchema is that it seems designed to build a
row. In my use case I want the each cell in the row to have all possible
leaf of the structure including array of array of array etc. The
TupleSchema did not let me drill down those structures. So I decided to use
the MaterializedField directly since then I could navigate down to the
leafs.


Interesting the Result Set Loader will have a dynamic batch size. I like
that. I've been wondering what to set my msgpack batch size too.


On Thu, Oct 11, 2018 at 1:52 AM Paul Rogers 
wrote:

> Hi JC,
>
> Drill's complex types can be a bit confusing. Note that, in your example,
> for the REPEATED BIGINT, we know that this is an array (REPEATED) and we
> know the type of each element (BIGINT).
>
> But, that REPEATED LIST, it is a list of ... what? The element type is
> missing.
>
> This is not the only hole. The UNION type has a list of child types which
> tell you the types in the UNION. But, if the UNION's child type is a MAP,
> that type does not include the full MAP schema.  A LIST is a list of a
> single type, or a LIST of UNIONs. It has the same schema ambiguity.
>
> The TupleSchema mechanism fills in this gap. But, for your use,
> MaterializedList should be fine because you probably don't want to use the
> "obscure" types. Frankly, LIST, REPEATED LIST and UNION are still pretty
> broken. I would recommend sticking with the scalar types (REQUIRED,
> OPTIONAL, REPEATED) and MAP (REQUIRED and REPEATED, there is no OPTIONAL).
> I ran into bug after bug when trying to use LIST or UNION. You can populate
> them, but some of the "fancier" operators (sort, hash join, aggregation)
> can't handle them yet.
>
>
> Can you explain a bit more the problem you ran into with the SchemaBuilder
> (the one that uses TupleSchema)? It is supposed to handle all types. I'd
> like to fix any issues you may have found.
>
>
> Just to give a bit more background on the tuple schema and related
> classes... The builder creates a schema that can be used with the RowSet
> class to create a record batch that matches the schema. The RowSet provides
> column writers to populate your record batch, and column readers to read
> it. The column accessors convert between Java types and vector types and
> can provide the custom type conversion I mentioned.
>
> For simple cases (working with a few types), the simple mechanism shown in
> the log reader works well. (It is what we explain in the Drill book.) But,
> as you add types, especially structured types, things get pretty complex.
> The RowSet family handles all that cruft for you.
>
> The part I still need to add is the "result set loader" which goes one
> step further: it can limit memory taken by a record batch. Most readers
> today use a fixed number, say 4K records. 4K of INTs is pretty small. 4K of
> 1 MB images is pretty big. The Result Set Loader works against a memory
> limit (20 MB, say) and automatically limits records per batch to that
> memory limit.
>
>
> Thanks for doing the PR. Will be great to see what you've created.
>
> Thanks,
> - Paul
>
>
>
> On Wednesday, October 10, 2018, 7:59:06 PM PDT, Jean-Claude Cote <
> jcc...@gmail.com> wrote:
>
>  Hey Paul,
>
> You mentionned that
>
> "But, for a LIST, the Materialized field does not include the child types"
>
> However MaterializedField do have type information for child types. You can
> see it in this example. I think it has all relevant information. Anyways
> all test cases I've tried so far are working..
>
>
> child {
>   major_type {
> 

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

2018-10-11 Thread GitBox
Ben-Zvi 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_r224639518
 
 

 ##
 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:
   How about `if 

[GitHub] kkhatua commented on issue #1491: DRILL-6084: Show Drill functions in WebUI for autocomplete

2018-10-11 Thread GitBox
kkhatua commented on issue #1491: DRILL-6084: Show Drill functions in WebUI for 
autocomplete
URL: https://github.com/apache/drill/pull/1491#issuecomment-429128624
 
 
   @arina-ielchiieva  / @ilooner  can one of you do a review of this minor 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] kkhatua closed pull request #1483: DRILL-3988: Expose Drill built-in functions & UDFs in a system table

2018-10-11 Thread GitBox
kkhatua closed pull request #1483: DRILL-3988: Expose Drill built-in functions 
& UDFs  in a system table
URL: https://github.com/apache/drill/pull/1483
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
index f4b83736739..a6f9a7f7ca7 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -28,6 +28,7 @@
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -54,6 +55,7 @@
 import org.apache.drill.exec.exception.FunctionValidationException;
 import org.apache.drill.exec.exception.JarValidationException;
 import org.apache.drill.exec.expr.fn.registry.LocalFunctionRegistry;
+import org.apache.drill.exec.expr.fn.registry.FunctionHolder;
 import org.apache.drill.exec.expr.fn.registry.JarScan;
 import org.apache.drill.exec.expr.fn.registry.RemoteFunctionRegistry;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
@@ -483,6 +485,17 @@ private ScanResult scan(ClassLoader classLoader, Path 
path, URL[] urls) throws I
 return missingJars;
   }
 
+  /**
+   * Retrieve all functions, mapped by source jars (after syncing)
+   * @return Map of source jars and their functionHolders
+   */
+  public Map> getAllJarsWithFunctionsHolders() {
+if (useDynamicUdfs) {
+  syncWithRemoteRegistry(localFunctionRegistry.getVersion());
+}
+return localFunctionRegistry.getAllJarsWithFunctionsHolders();
+  }
+
   /**
* Creates local udf directory, if it doesn't exist.
* Checks if local udf directory is a directory and if current application 
has write rights on it.
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolder.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolder.java
index d1d4fc94dfd..d0383e91fc8 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolder.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolder.java
@@ -25,6 +25,8 @@
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 
 import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
@@ -161,6 +163,38 @@ public void removeJar(String jarName) {
 }
   }
 
+  /**
+   * Retrieves all functions (holders) associated with all the jars
+   * This is read operation, so several users can perform this operation at 
the same time.
+   * @return list of all functions, mapped by their sources
+   */
+  public Map> getAllJarsWithFunctionHolders() {
+Map> allFunctionHoldersByJar = new 
HashMap<>();
+
+try (@SuppressWarnings("unused") Closeable lock = readLock.open()) {
+  for (String jarName : jars.keySet()) {
+//Capture functionHolders here
+List drillFuncHolderList = new LinkedList<>();
+
+Map> functionsInJar = jars.get(jarName);
+for (Map.Entry> functionEntry : 
functionsInJar.entrySet()) {
+  String fnName = functionEntry.getKey();
+  Queue fnSignatureList = functionEntry.getValue();
+  //Get all FunctionHolders (irrespective of source)
+  Map functionHolders = functions.get(fnName);
+  //Iterate for matching entries and populate new Map
+  for (Map.Entry entry : 
functionHolders.entrySet()) {
+if (fnSignatureList.contains(entry.getKey())) {
+  drillFuncHolderList.add(new FunctionHolder(fnName, 
entry.getKey(), entry.getValue()));
+}
+  }
+}
+allFunctionHoldersByJar.put(jarName, drillFuncHolderList);
+  }
+}
+return allFunctionHoldersByJar;
+  }
+
   /**
* Retrieves all function names associated with the jar from {@link #jars}.
* Returns empty list if jar is not registered.
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
index cefbd8cf388..f96b1fbb6ed 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
@@ -235,6 +235,14 @@ 

[GitHub] kkhatua commented on a change in pull request #1483: DRILL-3988: Expose Drill built-in functions & UDFs in a system table

2018-10-11 Thread GitBox
kkhatua commented on a change in pull request #1483: DRILL-3988: Expose Drill 
built-in functions & UDFs  in a system table
URL: https://github.com/apache/drill/pull/1483#discussion_r224535223
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.java
 ##
 @@ -155,6 +162,47 @@ public void testGetAllJarNames() {
 compareTwoLists(expectedResult, registryHolder.getAllJarNames());
   }
 
+  @Test
+  public void testGetAllJarsWithFunctionHolders() {
+Map> fnHoldersInRegistry = 
registryHolder.getAllJarsWithFunctionHolders();
+//Iterate and confirm lists are same
+for (String jarName : newJars.keySet()) {
+  List expectedHolderList = newJars.get(jarName).stream()
+  .map(FunctionHolder::getHolder) //Extract DrillFuncHolder
+  .collect(Collectors.toList());
+  List testHolderList = 
fnHoldersInRegistry.get(jarName).stream()
+  .map(FunctionHolder::getHolder) //Extract DrillFuncHolder
+  .collect(Collectors.toList());
+
+  compareTwoLists(expectedHolderList, testHolderList);
+}
+
+Map shuffleFunctionMap = new HashMap<>();
+// Confirm that same function spans multiple jars with different signatures
+//Init: Expected Map of items
+for (String jarName : newJars.keySet()) {
+  for (FunctionHolder funcHolder : newJars.get(jarName)) {
+if (funcHolder.getName().equals(SHUFFLE_FUNC_NAME)) {
 
 Review comment:
    


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


[jira] [Created] (DRILL-6790) Updating SET and ALTER SYSTEM doc pages

2018-10-11 Thread Vitalii Diravka (JIRA)
Vitalii Diravka created DRILL-6790:
--

 Summary: Updating SET and ALTER SYSTEM doc pages
 Key: DRILL-6790
 URL: https://issues.apache.org/jira/browse/DRILL-6790
 Project: Apache Drill
  Issue Type: Task
  Components: Documentation
Affects Versions: 1.14.0
Reporter: Vitalii Diravka
 Fix For: 1.15.0


It is necessary to update old queries related to "sys.options" table in SET 
[https://drill.apache.org/docs/set/] and ALTER SYSTEM 
[https://drill.apache.org/docs/alter-system/] drill  doc pages:
{code}
0: jdbc:drill:zk=local> SELECT name, type, bool_val FROM sys.options WHERE type 
= 'SESSION' order by name;
18:04:32.669 [2440997e-a5e7-4458-b7f9-661df830db5f:foreman] ERROR 
o.a.calcite.runtime.CalciteException - 
org.apache.calcite.sql.validate.SqlValidatorException: Column 'type' not found 
in any table
18:04:32.670 [2440997e-a5e7-4458-b7f9-661df830db5f:foreman] ERROR 
o.a.calcite.runtime.CalciteException - 
org.apache.calcite.runtime.CalciteContextException: From line 1, column 52 to 
line 1, column 55: Column 'type' not found in any table
Error: VALIDATION ERROR: From line 1, column 52 to line 1, column 55: Column 
'type' not found in any table

[Error Id: ffb96168-50e6-4d38-baa1-09429f3f191b on vitalii-pc:31010] 
(state=,code=0)
{code}

DRILL-5723 introduces new fields for "sys.options" system table.
The "type" field could be replaced with "optionScope":
{code}
0: jdbc:drill:zk=local> SELECT * FROM sys.options WHERE optionScope in 
('SYSTEM','SESSION') order by name;
+-+---+---++--+--+-+
|  name   | kind  | accessibleScopes  | 
 val   |  status  | optionScope  | description  
   |
+-+---+---++--+--+-+
| drill.exec.functions.cast_empty_string_to_null  | BIT   | ALL   | 
true   | CHANGED  | SESSION  | In a text file, treat empty fields as NULL 
values instead of empty string.  |
| drill.exec.hashagg.fallback.enabled | BIT   | ALL   | 
false  | DEFAULT  | SYSTEM   |  
   |
+-+---+---++--+--+-+
2 rows selected (0.113 seconds)
{code}




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


[jira] [Resolved] (DRILL-3214) Config option to cast empty string to null does not cast empty string to null

2018-10-11 Thread Vitalii Diravka (JIRA)


 [ 
https://issues.apache.org/jira/browse/DRILL-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka resolved DRILL-3214.

   Resolution: Won't Fix
 Assignee: Vitalii Diravka  (was: Sean Hsuan-Yi Chu)
Fix Version/s: (was: 1.1.0)
   1.0.0

The option doesn't work on SESSION level. It should be resolved in DRILL-3259:
{code}
0: jdbc:drill:zk=local> alter session set 
`drill.exec.functions.cast_empty_string_to_null` = true;
+---+--+
|  ok   | summary  |
+---+--+
| true  | drill.exec.functions.cast_empty_string_to_null updated.  |
+---+--+
1 row selected (0.064 seconds)
0: jdbc:drill:zk=local> SELECT cast('' as int) FROM (VALUES(1));
Error: SYSTEM ERROR: NumberFormatException: 

[Error Id: 4e7f8752-ec87-47c4-8ac9-15850bbd4c70 on vitalii-pc:31010] 
(state=,code=0)
{code}

But the option works on SYSTEM level:
{code}
[Error Id: 58265ead-b835-4772-89b9-1a90bce3c7e0 on vitalii-pc:31010] 
(state=,code=0)
0: jdbc:drill:zk=local> alter system set 
`drill.exec.functions.cast_empty_string_to_null` = true;
+---+--+
|  ok   | summary  |
+---+--+
| true  | drill.exec.functions.cast_empty_string_to_null updated.  |
+---+--+
1 row selected (0.08 seconds)
0: jdbc:drill:zk=local> SELECT cast('' as int) FROM (VALUES(1));
+-+
| EXPR$0  |
+-+
| null|
+-+
1 row selected (0.094 seconds)
{code}

[~mrsarm] Could you double check it? If your case differs, feel free to create 
another ticket to solve that.

> Config option to cast empty string to null does not cast empty string to null
> -
>
> Key: DRILL-3214
> URL: https://issues.apache.org/jira/browse/DRILL-3214
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Functions - Drill
>Affects Versions: 1.0.0
> Environment: faec150598840c40827e6493992d81209aa936da
>Reporter: Khurram Faraaz
>Assignee: Vitalii Diravka
>Priority: Major
> Fix For: 1.0.0
>
>
> Config option drill.exec.functions.cast_empty_string_to_null does not seem to 
> be working as designed.
> Disable casting of empty strings to null. 
> {code}
> 0: jdbc:drill:schema=dfs.tmp> alter session set 
> `drill.exec.functions.cast_empty_string_to_null` = false;
> +---+--+
> |  ok   | summary  |
> +---+--+
> | true  | drill.exec.functions.cast_empty_string_to_null updated.  |
> +---+--+
> 1 row selected (0.078 seconds)
> {code}
> In this query we see empty strings are retained in query output in columns[1].
> {code}
> 0: jdbc:drill:schema=dfs.tmp> SELECT columns[0], columns[1], columns[2] from 
> `threeColsDouble.csv`;
> +--+-+-+
> |  EXPR$0  | EXPR$1  | EXPR$2  |
> +--+-+-+
> | 156  | 234 | 1   |
> | 2653543  | 434 | 0   |
> | 367345   | 567567  | 23  |
> | 34554| 1234| 45  |
> | 4345 | 567678  | 19876   |
> | 34556| 0   | 1109|
> | 5456 | -1  | 1098|
> | 6567 | | 34534   |
> | 7678 | 1   | 6   |
> | 8798 | 456 | 243 |
> | 265354   | 234 | 123 |
> | 367345   | | 234 |
> | 34554| 1   | 2   |
> | 4345 | 0   | 10  |
> | 34556| -1  | 19  |
> | 5456 | 23423   | 345 |
> | 6567 | 0   | 2348|
> | 7678 | 1   | 2   |
> | 8798 | | 45  |
> | 099  | 19  | 17  |
> +--+-+-+
> 20 rows selected (0.13 seconds)
> {code}
> Casting empty strings to integer leads to NumberFormatException
> {code}
> 0: jdbc:drill:schema=dfs.tmp> SELECT columns[0], cast(columns[1] as int), 
> columns[2] from `threeColsDouble.csv`;
> Error: SYSTEM ERROR: java.lang.NumberFormatException: 
> Fragment 0:0
> [Error Id: b08f4247-263a-460d-b37b-91a70375f7ba on centos-03.qa.lab:31010] 
> (state=,code=0)
> {code}
> Enable casting empty string to null.
> {code}
> 0: jdbc:drill:schema=dfs.tmp> alter session set 
> `drill.exec.functions.cast_empty_string_to_null` = true;
> +---+--+
> |  ok   | summary

[jira] [Resolved] (DRILL-3363) Give user exception when users are trying to set system-wide-variable with "alter session set"

2018-10-11 Thread Vitalii Diravka (JIRA)


 [ 
https://issues.apache.org/jira/browse/DRILL-3363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka resolved DRILL-3363.

   Resolution: Won't Fix
 Assignee: Vitalii Diravka
Fix Version/s: (was: Future)
   1.12.0

The reason of DRILL-3259 is other. I have described it there in comment.
The exception is thrown for the case of setting SYSTEM-wide option on the 
SESSION level. It was resolved in the DRILL-5723:
{code}
0: jdbc:drill:zk=local> alter session set `exec.queue.memory_reserve_ratio`=0.5;
Error: PERMISSION ERROR: Cannot change option exec.queue.memory_reserve_ratio 
in scope SESSION

[Error Id: 7ce8015f-7db0-4486-967c-23f6623b343b on vitalii-pc:31010] 
(state=,code=0)
{code}

> Give user exception when users are trying to set system-wide-variable with 
> "alter session set"
> --
>
> Key: DRILL-3363
> URL: https://issues.apache.org/jira/browse/DRILL-3363
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Sean Hsuan-Yi Chu
>Assignee: Vitalii Diravka
>Priority: Major
> Fix For: 1.12.0
>
>
> When user tries to set a system-wide variable by alter session, a 
> warning/error message should pop up. 
> Otherwise, issues such as DRILL-3214 will ensue. 



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


[jira] [Resolved] (DRILL-5192) REFRESH table METADATA as default system option

2018-10-11 Thread Vitalii Diravka (JIRA)


 [ 
https://issues.apache.org/jira/browse/DRILL-5192?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka resolved DRILL-5192.

Resolution: Won't Fix
  Assignee: Vitalii Diravka

[~bgajjela] Once you perform the query and the data is changed after last 
{{REFRESH TABLE}}, the metadata cache files are regenerated automatically. No 
need to perform {{REFRESH METADATA TABLE}}.
 See details in 
[docs|https://drill.apache.org/docs/optimizing-parquet-metadata-reading/#generating-the-parquet-metadata-cache-file]
 and 
[code|https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java#L642].

Also the new "Metadata Management" work aka "Drill Metastore" is in progress. 
If you have suggestions, please put them in DRILL-6552

> REFRESH table METADATA as default system option 
> 
>
> Key: DRILL-5192
> URL: https://issues.apache.org/jira/browse/DRILL-5192
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.5.0, 1.6.0, 1.7.0, 1.8.0
> Environment: maprfs
>Reporter: bharath kumar
>Assignee: Vitalii Diravka
>Priority: Major
>
> Hi,
> We have a batch use case where files are loaded daily , we run REFRESH 
> METADATA TABLE; for more performance on these files. How can we automate this 
> refresh metadata table; at drill configuration level for specific table.
> Thanks,
> Bharath



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


[GitHub] vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428948476
 
 
   I don't think that this is a good idea to add an option. In the case when we 
have a combination of these two problems, option won't help.
   
   I think it is possible to enhance the way for collecting setter methods for 
example to map full [nested] class names and its setters, so after that receive 
nested class instances and call setters for them in 
`CodeGenMemberInjector.injectMembers()`.


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] jcmcote opened a new pull request #1500: Msgpack format reader

2018-10-11 Thread GitBox
jcmcote opened a new pull request #1500: Msgpack format reader
URL: https://github.com/apache/drill/pull/1500
 
 
   Implementation of a msgpack format reader
   
   - schema learning
   - skip over malformed records
   - skip over invalid field names
   - skip over records not matching schema
   - writing msgpack has not yet been implemented
   
   implementation of a zstandard codec
   - only decompression is implemented 
   


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] vdiravka commented on a change in pull request #1494: DRILL-6768: Improve to_date, to_time and to_timestamp and correspondi…

2018-10-11 Thread GitBox
vdiravka commented on a change in pull request #1494: DRILL-6768: Improve 
to_date, to_time and to_timestamp and correspondi…
URL: https://github.com/apache/drill/pull/1494#discussion_r223341697
 
 

 ##
 File path: exec/java-exec/src/main/codegen/data/Casts.tdd
 ##
 @@ -61,10 +61,30 @@
 {from: "VarChar", to: "TimeStamp", major: "VarCharDate", alias: 
"timestamptype"},
 {from: "VarChar", to: "Time", major: "VarCharDate", alias: "timetype"},
 
+{from: "VarChar", to: "NullableDate", major: 
"EmptyStringVarCharDateComplex"},
+{from: "VarChar", to: "NullableTimeStamp", major: 
"EmptyStringVarCharDateComplex"},
+{from: "VarChar", to: "NullableTime", major: 
"EmptyStringVarCharDateComplex"},
+
+{from: "NullableVarChar", to: "NullableDate", major: 
"EmptyStringVarCharDateComplex"},
+{from: "NullableVarChar", to: "NullableTimeStamp", major: 
"EmptyStringVarCharDateComplex"},
+{from: "NullableVarChar", to: "NullableTime", major: 
"EmptyStringVarCharDateComplex"},
 
 Review comment:
   * Looks like I have introduced not good major name here a lot time ago, 
please replace all `EmptyStringVarChar` to `NullableVarChar`. 
   * What does `DateComplex` mean? Looks like just `Date` fully reflect that 
target types of converting.


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] lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428927289
 
 
   I agree with you, it will cause the overflow of the class constant pool in 
extreme case, that is to say a projection or a filter contains thousands (or 
maybe larger) of constants
   
   The good news is all tests in `TestLargeFileCompilation ` are succeeded with 
this change, shall we add an option to disable this functionality or maybe 
there is a good way to avoid?
   
   I'd do some tests to find out how much constants will cause the overflow of 
the class constant pool.
   


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] vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428920593
 
 
   @lushuifeng, then this change will break code compilation for the case when 
a lot of constants were used.
   
   Previously, these constants were separated between the outer and nested 
classes.
   But with these changes, additional items in the class constant pool will be 
used for setter methods[1], and in the outer class will be present all setters, 
so it will cause the overflow of the class constant pool.
   
   [1] https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-4.html#jvms-4.6


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


Re: [DISCUSSION] CI for Drill

2018-10-11 Thread Vitalii Diravka
Hi all!

I have opened PR with adding CircleCI configs for Drill build [1].
And the ticket [2] for INFRA to setup CircleCI for ApacheDrill.
But then I've noticed that INFRA can't allow write access for 3d party
(Apache Arrow + CircleCI [3]).
So here are two ways:
* to merge it, then CircleCI builds will work for Drill forks only.
* try to help INFRA to enable CircleCI for Apache Drill main repo via
configuring CircleCI webhooks [4]

I think we can proceed with both of them, since even just to merge
.circleci to the Drill will be useful for the forks
of committers and contributors (like in the Apache Cassandra [5]).
Thoughts?

[1] https://github.com/apache/drill/pull/1493
[2] https://issues.apache.org/jira/browse/INFRA-17133
[3]
https://issues.apache.org/jira/browse/INFRA-15964?focusedCommentId=16351422=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16351422
[4]
https://issues.apache.org/jira/browse/INFRA-12197?focusedCommentId=15652850=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15652850
[5] https://github.com/apache/cassandra/tree/trunk/.circleci

On Wed, Sep 12, 2018 at 3:41 PM Vitalii Diravka 
wrote:

> The current issue with CircleCI is default RAM limit of medium (default)
> instance for Docker images - 4Gb [1].
> It can be expanded by using VM instead of Docker image or possibly
> CircleCI team can provide us bigger instance for it [2]
>
> I have created the Jira ticket for it [3]. Further discussion can be
> continued there.
>
> [1] https://circleci.com/docs/2.0/configuration-reference/#resource_class
> [2] https://circleci.com/pricing/
> [3] https://issues.apache.org/jira/browse/DRILL-6741
>
> Kind regards
> Vitalii
>
>
> On Wed, Sep 12, 2018 at 12:27 PM Arina Yelchiyeva <
> arina.yelchiy...@gmail.com> wrote:
>
>> +1, especially if other Apache project uses it, there should not be any
>> issues with Apache.
>>
>> Kind regards,
>> Arina
>>
>> On Wed, Sep 12, 2018 at 12:36 AM Timothy Farkas  wrote:
>>
>> > +1 For trying out Circle CI. I've used it in the past, and I think the
>> UI
>> > is much better than Travis.
>> >
>> > Tim
>> >
>> > On Tue, Sep 11, 2018 at 8:21 AM Vitalii Diravka <
>> vitalii.dira...@gmail.com
>> > >
>> > wrote:
>> >
>> > > Recently we discussed Travis build failures and there were excluded
>> more
>> > > tests to make Travis happy [1]. But looks like the issue returned back
>> > and
>> > > Travis build fails intermittently.
>> > >
>> > > I tried to find other solution instead of exclusion Drill unit tests
>> and
>> > > found other good CI - CircleCI [2]. Looks like this CI will allow to
>> run
>> > > all unit tests successfully.
>> > > And it offers good conditions for open-source projects [3] (even OS X
>> > > environment is available).
>> > > The example of Apache project, which uses this CI is Apache Cassandra
>> [4]
>> > >
>> > > My quick set-up of CircleCI for Drill still fails, but it should be
>> just
>> > > configured properly [5].
>> > >
>> > > I think we can try CircleCI in parallel with Travis and if it works
>> well,
>> > > we will move completely to CircleCI.
>> > > Does it make sense? Maybe somebody faced with it and knows some
>> > limitations
>> > > or complexities?
>> > >
>> > > [1]
>> > >
>> >
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_DRILL-2D6559=DwIBaQ=cskdkSMqhcnjZxdQVpwTXg=4eQVr8zB8ZBff-yxTimdOQ=Q-8LHY-5W3frk1S48j4jsEgmHOKPowwFtsEHM9Fp_g4=0oL67ROsJWhMDYzDS-y3Ch-ibgsfKQph8tN0I0jsB1o=
>> > > [2]
>> > >
>> > >
>> >
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_DefinitelyTyped_DefinitelyTyped_issues_20308-23issuecomment-2D342115544=DwIBaQ=cskdkSMqhcnjZxdQVpwTXg=4eQVr8zB8ZBff-yxTimdOQ=Q-8LHY-5W3frk1S48j4jsEgmHOKPowwFtsEHM9Fp_g4=k1Q44t4uWwCoA0fUVtaoKHaXEMq4Gtf97k0ST1YjGNs=
>> > > [3]
>> > >
>> >
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__circleci.com_pricing_=DwIBaQ=cskdkSMqhcnjZxdQVpwTXg=4eQVr8zB8ZBff-yxTimdOQ=Q-8LHY-5W3frk1S48j4jsEgmHOKPowwFtsEHM9Fp_g4=2XHpHg1fhBVMrNA2HuZJCWl08PQ3SqJ0r0Kd3L9wqao=
>> > > [4]
>> > >
>> >
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_cassandra_blob_trunk_.circleci_config.yml=DwIBaQ=cskdkSMqhcnjZxdQVpwTXg=4eQVr8zB8ZBff-yxTimdOQ=Q-8LHY-5W3frk1S48j4jsEgmHOKPowwFtsEHM9Fp_g4=RiP35johSh3iM0LkqEDGuuMH_F9Hy4LBrtFOqCcTYQ4=
>> > > [5]
>> > >
>> >
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__circleci.com_gh_vdiravka_drill_tree_circleCI=DwIBaQ=cskdkSMqhcnjZxdQVpwTXg=4eQVr8zB8ZBff-yxTimdOQ=Q-8LHY-5W3frk1S48j4jsEgmHOKPowwFtsEHM9Fp_g4=TAY_BXixKtv88mMRkXzSFcIlJ5bYxigcAK0RbJsFlPU=
>> > >
>> > > Kind regards
>> > > Vitalii
>> > >
>> >
>>
>


[GitHub] vdiravka commented on issue #1493: DRILL-6777: Setup CircleCI configs for Drill

2018-10-11 Thread GitBox
vdiravka commented on issue #1493: DRILL-6777: Setup CircleCI configs for Drill
URL: https://github.com/apache/drill/pull/1493#issuecomment-428915471
 
 
   @ilooner Thanks for review.
   1. I have created 
[INFRA-17133](https://issues.apache.org/jira/browse/INFRA-17133) ticket, but 
then I've noticed that INFRA can't allow write access for 3d party ([Apache 
Arrow + 
CircleCI](https://issues.apache.org/jira/browse/INFRA-15964?focusedCommentId=16351422=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16351422)).
 So here are two ways:
   * merge it and CircleCI builds will work for Drill forks only.
   * try to help INFRA to solve this issue via configuring CircleCI webhooks 
[[link](https://issues.apache.org/jira/browse/INFRA-12197?focusedCommentId=15652850=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15652850)]
   
  I think we can try the second way, but even just to merge .circleci to 
the Drill will be useful for the forks 
  of committers and contributors, like in the Apache Cassandra 
[[link](https://github.com/apache/cassandra/tree/trunk/.circleci)].
  I will post this question to our community to decide.
   
   2. The status badge for README.md could be added once CirlceCI will be 
enabled for Drill.  https://issues.apache.org/jira/browse/DRILL-6789
   
   3. The ticket for caching already exists 
[DRILL-6780](https://issues.apache.org/jira/browse/DRILL-6780). There are also 
other future enhancements in 
[DRILL-6741](https://issues.apache.org/jira/browse/DRILL-6741), but even 
without them current Circle build runs faster with more tests than in Travis 
build.
   
   
   
   


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] lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428915244
 
 
   Both top and nested classes will be added since the new class member is in 
inner class if there is a inner class, the inner class member is initialized by 
invoking top class member function:
   
   top:
   `public void setConstant5(IntHolder intHolder) {
 this.innerClassField.setConstant5(intHolder)
   }`
   inner:
   `
   public void setConstant5(IntHolder intHolder) {
 this.constant5  = intHolder;
   }`
   


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


[jira] [Created] (DRILL-6789) Adding a Circle CI build status badge to the README.md

2018-10-11 Thread Vitalii Diravka (JIRA)
Vitalii Diravka created DRILL-6789:
--

 Summary: Adding a Circle CI build status badge to the README.md
 Key: DRILL-6789
 URL: https://issues.apache.org/jira/browse/DRILL-6789
 Project: Apache Drill
  Issue Type: Sub-task
  Components: Tools, Build  Test
Affects Versions: 1.14.0
Reporter: Vitalii Diravka
 Fix For: Future


Once CircleCi will work for Apache Drill the status badge should be added to 
the README.md file
https://circleci.com/gh/apache/drill/edit#badges



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


[GitHub] vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
vvysotskyi commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428905684
 
 
   @lushuifeng, I have a question regarding nested splitting classes: these 
setters will be added only to the top level class, or they will be separated 
between both: top and nested 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] arina-ielchiieva commented on issue #1307: DRILL-6473: Update MapR Hive, MapR release version and ojai version

2018-10-11 Thread GitBox
arina-ielchiieva commented on issue #1307: DRILL-6473: Update MapR Hive, MapR 
release version and ojai version
URL: https://github.com/apache/drill/pull/1307#issuecomment-428903561
 
 
   @Agirish looks like we should not merge this PR until 
https://github.com/apache/drill/pull/1489 is resolved.


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] lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL functions with constant values

2018-10-11 Thread GitBox
lushuifeng commented on issue #1481: DRILL-6763: Codegen optimization of SQL 
functions with constant values
URL: https://github.com/apache/drill/pull/1481#issuecomment-428900022
 
 
   @vvysotskyi Could you please take a look at these changes?
   > commit 8b44957 fix the compilation error if there are nested splitting 
classes
   > commit 5edecc4 and 413891a fix the compilation error of the innerClasses 
like BatchHolder and OutgoingRecordBatch contain constant values
   
   I've just found that previous errors in my tests `mvn clean install` are 
because of there is [a DOT in my user 
name](https://issues.apache.org/jira/browse/HADOOP-7050)
   


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] arina-ielchiieva closed pull request #1495: DRILL-6776: Drill Web UI takes long time for first time load in network isolated environment

2018-10-11 Thread GitBox
arina-ielchiieva closed pull request #1495: DRILL-6776: Drill Web UI takes long 
time for first time load in network isolated environment
URL: https://github.com/apache/drill/pull/1495
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/exec/java-exec/src/main/resources/rest/generic.ftl 
b/exec/java-exec/src/main/resources/rest/generic.ftl
index 81204f795c1..5b2dc45568c 100644
--- a/exec/java-exec/src/main/resources/rest/generic.ftl
+++ b/exec/java-exec/src/main/resources/rest/generic.ftl
@@ -35,9 +35,7 @@
 
   
 
-  
-   window.jQuery  || document.write('