You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2015/09/16 02:16:28 UTC

[1/2] drill git commit: DRILL-3680: Fix incorrect query result or IOBE when window function is used in subquery.

Repository: drill
Updated Branches:
  refs/heads/master 48bc0b9a8 -> 9afcf61f6


DRILL-3680: Fix incorrect query result or IOBE when window function is used in subquery.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/9afcf61f
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/9afcf61f
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/9afcf61f

Branch: refs/heads/master
Commit: 9afcf61f6c993cd028022d827daa7f873a61ffaa
Parents: 80e7e61
Author: Jinfeng Ni <jn...@apache.org>
Authored: Tue Aug 25 10:56:25 2015 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Tue Sep 15 16:24:37 2015 -0700

----------------------------------------------------------------------
 .../exec/planner/common/DrillRelOptUtil.java    | 13 ++++++
 .../exec/planner/common/DrillWindowRelBase.java |  2 +-
 .../apache/drill/exec/TestWindowFunctions.java  | 46 ++++++++++++++++++++
 3 files changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9afcf61f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index 0a13b5b..733577e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -27,10 +27,12 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 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.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Pair;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
@@ -126,6 +128,17 @@ public abstract class DrillRelOptUtil {
     }
   }
 
+  /** Returns a rowType having all unique field name.
+   *
+   * @param rowType : input rowType
+   * @param typeFactory : type factory used to create a new row type.
+   * @return
+   */
+  public static RelDataType uniqifyFieldName(final RelDataType rowType, final RelDataTypeFactory typeFactory) {
+    return typeFactory.createStructType(RelOptUtil.getFieldTypeList(rowType),
+        SqlValidatorUtil.uniquify(rowType.getFieldNames()));
+  }
+
   /**
    * Returns whether the leading edge of a given array of expressions is
    * wholly {@link RexInputRef} objects with types and names corresponding

http://git-wip-us.apache.org/repos/asf/drill/blob/9afcf61f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWindowRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWindowRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWindowRelBase.java
index 272137d..f32e4c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWindowRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWindowRelBase.java
@@ -36,6 +36,6 @@ public class DrillWindowRelBase extends Window implements DrillRelNode {
       List<RexLiteral> constants,
       RelDataType rowType,
       List<Group> windows) {
-    super(cluster, traits, child, constants, rowType, windows);
+    super(cluster, traits, child, constants, DrillRelOptUtil.uniqifyFieldName(rowType, cluster.getTypeFactory()), windows);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9afcf61f/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
index b60f188..a31e13e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
@@ -810,4 +810,50 @@ public class TestWindowFunctions extends BaseTestQuery {
         .build()
         .run();
   }
+
+  @Test // DRILL-3679, DRILL-3680
+  public void testWindowFunInNestSubQ() throws Exception {
+    final String query =
+        " select n_nationkey , n_regionkey , " +
+        "        lead(n_regionkey) OVER ( PARTITION BY n_regionkey ORDER BY n_nationkey) lead_c2 " +
+        " FROM (SELECT n_nationkey ,n_regionkey, " +
+        "          ntile(3) over(PARTITION BY n_regionkey ORDER BY n_nationkey) " +
+        "       FROM cp.`tpch/nation.parquet`) " +
+        " order by n_regionkey, n_nationkey";
+    test(query);
+
+    final String baselineQuery =
+        "select n_nationkey , n_regionkey , " +
+        "       lead(n_regionkey) OVER ( PARTITION BY n_regionkey ORDER BY n_nationkey) lead_c2 " +
+        "FROM cp.`tpch/nation.parquet`   " +
+        "order by n_regionkey, n_nationkey";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .sqlBaselineQuery(baselineQuery)
+        .build()
+        .run();
+
+    final String query2 =
+         " select rnum, position_id, " +
+         "   ntile(4) over(order by position_id) " +
+         " from (select position_id, row_number() " +
+         "       over(order by position_id) as rnum " +
+         "       from cp.`employee.json`)";
+
+    final String baselineQuery2 =
+        " select row_number() over(order by position_id) as rnum, " +
+        "    position_id, " +
+        "    ntile(4) over(order by position_id) " +
+        " from cp.`employee.json`";
+
+    testBuilder()
+        .sqlQuery(query2)
+        .ordered()
+        .sqlBaselineQuery(baselineQuery2)
+        .build()
+        .run();
+  }
+
 }
\ No newline at end of file


[2/2] drill git commit: DRILL-3761: Modify CastIntDecimal implementation so that it will not modify its input holder.

Posted by jn...@apache.org.
DRILL-3761: Modify CastIntDecimal implementation so that it will not modify its input holder.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/80e7e61a
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/80e7e61a
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/80e7e61a

Branch: refs/heads/master
Commit: 80e7e61aa1396c0e424b42507cbf49c6b3546876
Parents: 48bc0b9
Author: Jinfeng Ni <jn...@apache.org>
Authored: Fri Aug 28 16:50:21 2015 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Tue Sep 15 16:24:37 2015 -0700

----------------------------------------------------------------------
 .../src/main/codegen/templates/Decimal/CastIntDecimal.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/80e7e61a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
index 2393db8..6d727b0 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -90,9 +90,10 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
         int scaleSize = org.apache.drill.exec.util.DecimalUtility.roundUp((int) scale.value);
         int integerIndex = (${type.arraySize} - scaleSize - 1);
 
-        while (in.value != 0 && integerIndex >= 0) {
-            out.setInteger(integerIndex--, (int) Math.abs((in.value % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE)), out.start, out.buffer);
-            in.value = in.value / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
+        long inValue = in.value;
+        while (inValue != 0 && integerIndex >= 0) {
+            out.setInteger(integerIndex--, (int) Math.abs((inValue % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE)), out.start, out.buffer);
+            inValue = inValue / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
         }
 
         </#if>