[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-18 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/drill/pull/1066


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-17 Thread amansinha100
Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r162171143
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java 
---
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, 
RelOptSchema relOptSchema) {
+super(context, cluster, relOptSchema);
+this.filterFactory =
+Util.first(context.unwrap(RelFactories.FilterFactory.class),
+RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate 
is created.
+   */
+  @Override
+  public RelBuilder empty() {
--- End diff --

Ok, thanks for clarifying.  The schema-on-read does make it difficult to 
avoid overriding the empty() behavior.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-17 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r162157833
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java 
---
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, 
RelOptSchema relOptSchema) {
+super(context, cluster, relOptSchema);
+this.filterFactory =
+Util.first(context.unwrap(RelFactories.FilterFactory.class),
+RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate 
is created.
--- End diff --

Thanks, fixed


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-17 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r162161040
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java 
---
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, 
RelOptSchema relOptSchema) {
+super(context, cluster, relOptSchema);
+this.filterFactory =
+Util.first(context.unwrap(RelFactories.FilterFactory.class),
+RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate 
is created.
+   */
+  @Override
+  public RelBuilder empty() {
--- End diff --

In some cases, RowType of the input may not be known at the moment when 
this method is called (some of the columns or all columns were not dynamically 
discovered yet), therefore I had to make changes directly in Calcite to allow 
using of custom RelBuilder and make changes in Drill to pass custom RelBuilder 
into them. For more details please see [1] and [2].

[1] 
https://lists.apache.org/list.html?d...@calcite.apache.org:lte=1y:Make%20RelBuilder.filter%28%29%20configurable
[2] https://issues.apache.org/jira/browse/CALCITE-2043



---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-17 Thread amansinha100
Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r157854723
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java 
---
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, 
RelOptSchema relOptSchema) {
+super(context, cluster, relOptSchema);
+this.filterFactory =
+Util.first(context.unwrap(RelFactories.FilterFactory.class),
+RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate 
is created.
+   */
+  @Override
+  public RelBuilder empty() {
--- End diff --

Is a separate DrillRelBuilder absolutely needed ?  Should the original 
behavior of empty() in Calcite be changed to preserve the RowType of the input 
? The fewer things we extend in Drill the better. 


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-17 Thread amansinha100
Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r157853167
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java 
---
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, 
RelOptSchema relOptSchema) {
+super(context, cluster, relOptSchema);
+this.filterFactory =
+Util.first(context.unwrap(RelFactories.FilterFactory.class),
+RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate 
is created.
--- End diff --

'data' spelling


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-16 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r161796052
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
 ---
@@ -1303,6 +1305,8 @@ private void checkGroupAndAggrValues(int 
incomingRowIdx) {
   long memDiff = allocator.getAllocatedMemory() - allocatedBeforeHTput;
   if ( memDiff > 0 ) { logger.warn("Leak: HashTable put() OOM left 
behind {} bytes allocated",memDiff); }
 
+  checkForSpillPossibility(currentPartition);
--- End diff --

These checks were needed to avoid infinite loop when there is not enough 
memory for the spill. 
I moved these checks into `spillIfNeeded()` method, so when called 
`doSpill()`, `forceSpill` in `spillIfNeeded()` is true and check should be done.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-15 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r161607926
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
 ---
@@ -1303,6 +1305,8 @@ private void checkGroupAndAggrValues(int 
incomingRowIdx) {
   long memDiff = allocator.getAllocatedMemory() - allocatedBeforeHTput;
   if ( memDiff > 0 ) { logger.warn("Leak: HashTable put() OOM left 
behind {} bytes allocated",memDiff); }
 
+  checkForSpillPossibility(currentPartition);
--- End diff --

Not sure this check 'chooseAPartitionToFlush'  is needed. If an exception 
is desired, I would think modifying doSpill() is better way e.g. modifying this 
line: "  if ( victimPartition < 0 ) { return; } " Otherwise in this process 
chooseAPartitionToFlush will be called twice.

  int victimPartition = chooseAPartitionToFlush(currentPartition, 
forceSpill);

  // In case no partition has more than one batch -- try and "push the 
limits"; maybe next
  // time the spill could work.
  if ( victimPartition < 0 ) { return; } 


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-12 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r161185413
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
 ---
@@ -82,4 +83,19 @@ protected boolean create2PhasePlan(RelOptRuleCall call, 
DrillAggregateRel aggreg
 }
 return true;
   }
+
+  /**
+   * Returns group-by keys with the remapped arguments for specified 
aggregate.
+   *
+   * @param groupSet ImmutableBitSet of aggregate rel node, whose group-by 
keys should be remapped.
+   * @return {@link ImmutableBitSet} instance with remapped keys.
+   */
+  public static ImmutableBitSet remapGroupSet(ImmutableBitSet groupSet) {
--- End diff --

After the changes in CALCITE-1930 in the class 
`AggregateExpandDistinctAggregatesRule`, this rule started applying more 
correctly, since in the older version there were checks like this:
```
aggCall.getAggregation() instanceof SqlCountAggFunction
```
but they were replaced by checks like this:
```
final SqlKind aggCallKind = aggCall.getAggregation().getKind();
switch (aggCallKind)
```
So for the cases when instead of Calcite s `SqlCountAggFunction` were used 
`DrillCalciteSqlAggFunctionWrapper`s this rule changed its behaviour and 
instead of returning joins of distinct and non-distinct aggregate rel nodes, it 
returns distinct aggregate which has an input with non-distinct aggregates 
grouped by column, which is distinct in outer aggregate.

These Drill rules were able to work correctly only for the cases when 
aggregate rel node does not contain ` aggCalls` and contains only the single 
field in `rowType` (in this case `groupSet` is always `{0}`, and it still 
correct for outer aggregates which are created in `*AggPrule`s)

With the new version of `AggregateExpandDistinctAggregatesRule` these Drill 
rules received aggregate rel nodes with the non-empty lists of ` aggCalls`, 
therefore its `rowType` contains more than one field. But before my changes, 
the same `groupSet` was passed to the constructor of outer aggregate and row 
type of aggregate differed from the row type of its input. So it was incorrect 
`groupSet`. 
Aggregate rel nodes always specify the group by columns in the first 
positions of the list, so correct `groupSet` for outer aggregate should be 
`ImmutableBitSet` with the same size as the `groupSet` of nested aggregate, but 
the ordinals of columns should start from 0. 

As for the point of iterating through the group set, 
`ImmutableBitSet.size()`, `ImmutableBitSet.length()` and 
`ImmutableBitSet.cardinality()` does not return desired "size" of the 
`groupSet`. `AggregateExpandDistinctAggregatesRule` also contains similar code 
which iterating through the `groupSet` for similar purposes.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2018-01-11 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r161077516
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
 ---
@@ -82,4 +83,19 @@ protected boolean create2PhasePlan(RelOptRuleCall call, 
DrillAggregateRel aggreg
 }
 return true;
   }
+
+  /**
+   * Returns group-by keys with the remapped arguments for specified 
aggregate.
+   *
+   * @param groupSet ImmutableBitSet of aggregate rel node, whose group-by 
keys should be remapped.
+   * @return {@link ImmutableBitSet} instance with remapped keys.
+   */
+  public static ImmutableBitSet remapGroupSet(ImmutableBitSet groupSet) {
--- End diff --

what is the reason we are going this remap with new calcite? 
And if the result is only depended on size of groupSet, we don't really 
need to iterate through the groupSet.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158374197
  
--- Diff: exec/java-exec/src/main/codegen/includes/parserImpls.ftl ---
@@ -351,4 +351,23 @@ SqlNode SqlDropFunction() :
{
return new SqlDropFunction(pos, jar);
}
-}
\ No newline at end of file
+}
+
+<#if !parser.includeCompoundIdentifier >
--- End diff --

Actually, it does not cause a new behaviour or functionality, it just helps 
to preserve old one after changes in Calcite `Parser.jj`. Therefore existing 
unit tests cover this change.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158335378
  
--- Diff: exec/java-exec/src/main/codegen/includes/parserImpls.ftl ---
@@ -351,4 +351,23 @@ SqlNode SqlDropFunction() :
{
return new SqlDropFunction(pos, jar);
}
-}
\ No newline at end of file
+}
+
+<#if !parser.includeCompoundIdentifier >
--- End diff --

do we need a test case for this newly added 
ParenthesizedCompoundIdentifierList?


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158259594
  
--- Diff: 
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
 ---
@@ -62,7 +62,7 @@ public JdbcPrel(RelOptCluster cluster, RelTraitSet 
traitSet, JdbcIntermediatePre
 (JavaTypeFactory) getCluster().getTypeFactory());
 final JdbcImplementor.Result result =
 jdbcImplementor.visitChild(0, input.accept(new SubsetRemover()));
-sql = result.asQuery().toSqlString(dialect).getSql();
+sql = result.asSelect().toSqlString(dialect).getSql();
--- End diff --

It is also may be another `SqlCall` instance. Thanks for pointing this, 
replaced it by `asStatement()` method.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158258451
  
--- Diff: 
contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
 ---
@@ -72,7 +72,16 @@ public AbstractSchema getSubSchema(String name) {
 @Override
 public Table getTable(String name) {
   HBaseScanSpec scanSpec = new HBaseScanSpec(name);
-  return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  try {
+return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  } catch (Exception e) {
+// Calcite firstly is looking for a table in the default schema, 
if a table was not found,
+// it is looking in root schema.
+// If a table does not exist, a query will fail at validation 
stage,
+// so the error should not be thrown there.
--- End diff --

Yes, I meant it. Thanks, replaced.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158258149
  
--- Diff: exec/java-exec/src/main/codegen/data/Parser.tdd ---
@@ -75,6 +72,26 @@
   implementationFiles: [
 "parserImpls.ftl"
   ]
+
+  # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+  # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+  createStatementParserMethods: [
--- End diff --

Calcites `Parser.jj` uses these lists to extend existing functionality when 
desired methods specified inside it. If we did not specify these lists, java 
class `Parser.java` could not be generated and build would fail with the error:
```
[ERROR] Failed to execute goal 
org.apache.drill.tools:drill-fmpp-maven-plugin:1.12.0-SNAPSHOT:generate 
(generate-fmpp) on project drill-java-exec: FMPP processing session failed.
[ERROR] Caused by: freemarker.core.InvalidReferenceException: The following 
has evaluated to null or missing:
[ERROR] ==> parser.createStatementParserMethods  [in template "Parser.jj" 
at line 881, column 6]
```


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158258329
  
--- Diff: 
contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
 ---
@@ -72,7 +72,16 @@ public AbstractSchema getSubSchema(String name) {
 @Override
 public Table getTable(String name) {
   HBaseScanSpec scanSpec = new HBaseScanSpec(name);
-  return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  try {
+return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  } catch (Exception e) {
+// Calcite firstly is looking for a table in the default schema, 
if a table was not found,
--- End diff --

Thanks, done.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158260235
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
 ---
@@ -470,34 +576,32 @@ public void disallowTemporaryTables() {
  * @throws UserException if temporary tables usage is disallowed
  */
 @Override
-public RelOptTableImpl getTable(final List names) {
-  RelOptTableImpl temporaryTable = null;
-
-  if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), 
drillConfig)) {
-String temporaryTableName = 
session.resolveTemporaryTableName(names.get(names.size() - 1));
-if (temporaryTableName != null) {
-  List temporaryNames = 
Lists.newArrayList(temporarySchema, temporaryTableName);
-  temporaryTable = super.getTable(temporaryNames);
+public Prepare.PreparingTable getTable(final List names) {
+  String originalTableName = 
session.getOriginalTableNameFromTemporaryTable(names.get(names.size() - 1));
+  if (originalTableName != null) {
+if (!allowTemporaryTables) {
+  throw UserException
+  .validationError()
+  .message("Temporary tables usage is disallowed. Used 
temporary table name: [%s].", originalTableName)
+  .build(logger);
 }
   }
-  if (temporaryTable != null) {
-if (allowTemporaryTables) {
-  return temporaryTable;
+  // Fix for select from hbase table with schema name in query 
(example: "SELECT col FROM hbase.t)
+  // from hbase schema (did "USE hbase" before).
--- End diff --

This change was made before the change, where we catch and log exceptions 
from `HBaseSchema.getTable()` method. So yes, it is not needed now.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158263816
  
--- Diff: exec/jdbc-all/pom.xml ---
@@ -572,7 +572,7 @@
   This is likely due to you adding new 
dependencies to a java-exec and not updating the excludes in this module. This 
is important as it minimizes the size of the dependency of Drill application 
users.
 
 
-2900
+3100
--- End diff --

We didn't have problems on our local machines or test cluster connected 
with `maxsize` after changing it to 3100. Which version of Maven did you 
use? 
Anyway, I have increased it to 3200.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-21 Thread vvysotskyi
Github user vvysotskyi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158261430
  
--- Diff: exec/java-exec/src/test/resources/record/test_recorditerator.json 
---
@@ -60,7 +60,7 @@
 @id:2,
 child:1,
 pop:"project",
-exprs:[ { ref : "`*`", expr : "`*`"} ]
+exprs:[ { ref : "`**`", expr : "`**`"} ]
--- End diff --

This change connected with commit `[CALCITE-1150] Add dynamic record type 
and dynamic star for schema-on-read table`. `**` is used as the dynamic star 
column name prefix. It is equivalent to `*` in our Drill-Calcite 1.4.0.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158173488
  
--- Diff: exec/java-exec/src/test/resources/record/test_recorditerator.json 
---
@@ -60,7 +60,7 @@
 @id:2,
 child:1,
 pop:"project",
-exprs:[ { ref : "`*`", expr : "`*`"} ]
+exprs:[ { ref : "`**`", expr : "`**`"} ]
--- End diff --

Not sure I understand this '**' thing, can you explain more about this 
change?


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158173904
  
--- Diff: exec/jdbc-all/pom.xml ---
@@ -572,7 +572,7 @@
   This is likely due to you adding new 
dependencies to a java-exec and not updating the excludes in this module. This 
is important as it minimizes the size of the dependency of Drill application 
users.
 
 
-2900
+3100
--- End diff --

I played with this branch and I have to change the size of 3100 to 
3200, it might due to my build environment but we may want to increase it 
to 3200.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158117382
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
 ---
@@ -470,34 +576,32 @@ public void disallowTemporaryTables() {
  * @throws UserException if temporary tables usage is disallowed
  */
 @Override
-public RelOptTableImpl getTable(final List names) {
-  RelOptTableImpl temporaryTable = null;
-
-  if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), 
drillConfig)) {
-String temporaryTableName = 
session.resolveTemporaryTableName(names.get(names.size() - 1));
-if (temporaryTableName != null) {
-  List temporaryNames = 
Lists.newArrayList(temporarySchema, temporaryTableName);
-  temporaryTable = super.getTable(temporaryNames);
+public Prepare.PreparingTable getTable(final List names) {
+  String originalTableName = 
session.getOriginalTableNameFromTemporaryTable(names.get(names.size() - 1));
+  if (originalTableName != null) {
+if (!allowTemporaryTables) {
+  throw UserException
+  .validationError()
+  .message("Temporary tables usage is disallowed. Used 
temporary table name: [%s].", originalTableName)
+  .build(logger);
 }
   }
-  if (temporaryTable != null) {
-if (allowTemporaryTables) {
-  return temporaryTable;
+  // Fix for select from hbase table with schema name in query 
(example: "SELECT col FROM hbase.t)
+  // from hbase schema (did "USE hbase" before).
--- End diff --

Could you explain why this is needed now? I think this used to work -- if a 
schema is not found under default, Drill falls back to the root to do the 
search. 

What got changed thus you have to introduce this fix? 

What about this test case?
"use hbase; select t.col, t2.col2 from hbase2.t2 as t2, hbase.t as t where 
t.id = t2.id"



---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158105572
  
--- Diff: 
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
 ---
@@ -62,7 +62,7 @@ public JdbcPrel(RelOptCluster cluster, RelTraitSet 
traitSet, JdbcIntermediatePre
 (JavaTypeFactory) getCluster().getTypeFactory());
 final JdbcImplementor.Result result =
 jdbcImplementor.visitChild(0, input.accept(new SubsetRemover()));
-sql = result.asQuery().toSqlString(dialect).getSql();
+sql = result.asSelect().toSqlString(dialect).getSql();
--- End diff --

Is the 'result' here guaranteed to be a SqlSelect?


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158120135
  
--- Diff: 
contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
 ---
@@ -72,7 +72,16 @@ public AbstractSchema getSubSchema(String name) {
 @Override
 public Table getTable(String name) {
   HBaseScanSpec scanSpec = new HBaseScanSpec(name);
-  return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  try {
+return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  } catch (Exception e) {
+// Calcite firstly is looking for a table in the default schema, 
if a table was not found,
--- End diff --

'is looking for' and 'is looking in' seems to be saying Calcite IS working 
in this line of code, but I think you meant that for the new version calcite, 
it 'looks for' something... so would like to get some rephrase here.


---


[GitHub] drill pull request #1066: DRILL-3993: Changes to support Calcite 1.15

2017-12-20 Thread chunhui-shi
Github user chunhui-shi commented on a diff in the pull request:

https://github.com/apache/drill/pull/1066#discussion_r158103733
  
--- Diff: 
contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
 ---
@@ -72,7 +72,16 @@ public AbstractSchema getSubSchema(String name) {
 @Override
 public Table getTable(String name) {
   HBaseScanSpec scanSpec = new HBaseScanSpec(name);
-  return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  try {
+return new DrillHBaseTable(schemaName, plugin, scanSpec);
+  } catch (Exception e) {
+// Calcite firstly is looking for a table in the default schema, 
if a table was not found,
+// it is looking in root schema.
+// If a table does not exist, a query will fail at validation 
stage,
+// so the error should not be thrown there.
--- End diff --

do you mean 'should not be thrown HERE'? The same for other places.


---