You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2016/06/19 00:03:44 UTC

[4/6] drill git commit: DRILL-2385: Count on complex objects failed with missing function implementation - added MapHolder, ListHolder; - added testCountComplexObjects() unit test.

DRILL-2385: Count on complex objects failed with missing function implementation - added MapHolder, ListHolder; - added testCountComplexObjects() unit test.


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

Branch: refs/heads/master
Commit: f86c4fa8eccf5585a9fc358b024ca1cbb194fe2b
Parents: 3209886
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Thu Jun 16 16:45:20 2016 +0000
Committer: Parth Chandra <pa...@apache.org>
Committed: Sat Jun 18 17:02:59 2016 -0700

----------------------------------------------------------------------
 .../src/main/codegen/data/CountAggrTypes.tdd    | 21 +++++++++++
 .../exec/resolver/ResolverTypePrecedence.java   |  2 ++
 .../drill/exec/resolver/TypeCastRules.java      |  2 ++
 .../exec/fn/impl/TestAggregateFunctions.java    | 34 ++++++++++++++++++
 .../test/resources/complex/json/complex.json    | 30 ++++++++++++++++
 .../resources/complex/json/repeated_list.json   |  2 +-
 .../complex/json/repeated_list_map.json         |  2 +-
 .../drill/exec/expr/holders/ListHolder.java     | 37 ++++++++++++++++++++
 .../drill/exec/expr/holders/MapHolder.java      | 32 +++++++++++++++++
 .../exec/expr/holders/RepeatedListHolder.java   | 20 +++++++++--
 .../exec/expr/holders/RepeatedMapHolder.java    | 25 +++++++++++--
 11 files changed, 201 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
index 53e25f7..aec73d1 100644
--- a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
@@ -2,38 +2,59 @@
   countFunctionsInput: [
     "Bit",
     "NullableBit",
+    "RepeatedBit",
     "Int",
     "NullableInt",
+    "RepeatedInt",
     "BigInt",
     "NullableBigInt",
+    "RepeatedBigInt",
     "Float4",
     "NullableFloat4",
+    "RepeatedFloat4",
     "Float8",
     "NullableFloat8",
+    "RepeatedFloat8",
     "Date",
     "NullableDate",
+    "RepeatedDate",
     "TimeStamp",
     "NullableTimeStamp",
+    "RepeatedTimeStamp",
     "Time",
     "NullableTime",
+    "RepeatedTime",
     "IntervalDay",
     "NullableIntervalDay",
+    "RepeatedIntervalDay",
     "IntervalYear",
     "NullableIntervalYear",
+    "RepeatedIntervalYear",
     "Interval",
     "NullableInterval",
+    "RepeatedInterval",
     "VarChar",
     "NullableVarChar",
+    "RepeatedVarChar",
     "VarBinary"
     "NullableVarBinary",
+    "RepeatedVarBinary"
     "Decimal9",
     "NullableDecimal9",
+    "RepeatedDecimal9",
     "Decimal18",
     "NullableDecimal18",
+    "RepeatedDecimal18",
     "Decimal28Sparse",
     "NullableDecimal28Sparse",
+    "RepeatedDecimal28Sparse",
     "Decimal38Sparse",
     "NullableDecimal38Sparse"
+    "RepeatedDecimal38Sparse",
+    "List"
+    "RepeatedList",
+    "Map"
+    "RepeatedMap"
   ]
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
index 8c602b3..a28c95a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
@@ -77,6 +77,8 @@ public class ResolverTypePrecedence {
     precedenceMap.put(MinorType.INTERVALDAY, i+= 2);
     precedenceMap.put(MinorType.INTERVALYEAR, i+= 2);
     precedenceMap.put(MinorType.INTERVAL, i+= 2);
+    precedenceMap.put(MinorType.MAP, i += 2);
+    precedenceMap.put(MinorType.LIST, i += 2);
     precedenceMap.put(MinorType.UNION, i += 2);
 
     MAX_IMPLICIT_CAST_COST = i;

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
index ae42937..8bb6c2a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
@@ -754,6 +754,8 @@ public class TypeCastRules {
     rule.add(MinorType.FIXEDBINARY);
     rules.put(MinorType.VARBINARY, rule);
 
+    rules.put(MinorType.MAP, Sets.newHashSet(MinorType.MAP));
+    rules.put(MinorType.LIST, Sets.newHashSet(MinorType.LIST));
     rules.put(MinorType.UNION, Sets.newHashSet(MinorType.UNION));
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index 0e558a7..d99eb00 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.fn.impl;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.PlanTestBase;
@@ -28,6 +29,7 @@ import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.List;
+import java.util.Map;
 
 public class TestAggregateFunctions extends BaseTestQuery {
 
@@ -525,4 +527,36 @@ public class TestAggregateFunctions extends BaseTestQuery {
     PlanTestBase.testPlanMatchingPatterns(sql, expectedPlan, excludedPatterns);
   }
 
+  @Test // DRILL-2385: count on complex objects failed with missing function implementation
+  public void testCountComplexObjects() throws Exception {
+    final String query = "select count(t.%s) %s from cp.`complex/json/complex.json` t";
+    Map<String, String> objectsMap = Maps.newHashMap();
+    objectsMap.put("COUNT_BIG_INT_REPEATED", "sia");
+    objectsMap.put("COUNT_FLOAT_REPEATED", "sfa");
+    // TODO: can be uncommented after fixing DRILL-4664
+    // objectsMap.put("COUNT_MAP_REPEATED", "soa");
+    // objectsMap.put("COUNT_MAP_REQUIRED", "oooi");
+    objectsMap.put("COUNT_LIST_REPEATED", "odd");
+    objectsMap.put("COUNT_LIST_OPTIONAL", "sia");
+
+    for (String object: objectsMap.keySet()) {
+      String optionSetting = "";
+      if (object.equals("COUNT_LIST_OPTIONAL")) {
+        // if `exec.enable_union_type` parameter is true then BIGINT<REPEATED> object is converted to LIST<OPTIONAL> one
+        optionSetting = "alter session set `exec.enable_union_type`=true";
+      }
+      try {
+        testBuilder()
+            .sqlQuery(query, objectsMap.get(object), object)
+            .optionSettingQueriesForTestQuery(optionSetting)
+            .unOrdered()
+            .baselineColumns(object)
+            .baselineValues(3L)
+            .go();
+      } finally {
+        test("ALTER SESSION RESET `exec.enable_union_type`");
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/test/resources/complex/json/complex.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/complex/json/complex.json b/exec/java-exec/src/test/resources/complex/json/complex.json
new file mode 100644
index 0000000..5e52f63
--- /dev/null
+++ b/exec/java-exec/src/test/resources/complex/json/complex.json
@@ -0,0 +1,30 @@
+{"sia":[1, 11, 101, 1001],
+ "sfa":[0.0, 1.01, 10.222, 10.0006789],
+ "soa":[{"in":1},{"in":1,"fl":1.12345}, {"in":1, "fl":10.12345, "nul":null}, {"in":1, "fl":10.6789, "nul":null, "bool":true, "str":"here is a string at row 1"}],
+ "oooi":{"oa":{"oab":{"oabc":1}}},
+ "odd": [
+    [[1],[],[3]],
+    [],
+    [[5]]
+ ]
+}
+{"sia":[2, 12, 102, 1002],
+ "sfa":[0.0, 2.01, 20.222, 20.0006789],
+ "soa":[{"in":2},{"in":2,"fl":2.12345}, {"in":2, "fl":20.12345, "nul":"not null"}, {"in":2, "fl":20.6789, "nul":null, "bool":false, "str":"here is a string at row 2"}],
+ "oooi":{"oa":{"oab":{"oabc":2}}},
+ "odd": [
+    [[1],[],[3]],
+    [],
+    [[5]]
+ ]
+}
+{"sia":[3, 13, 103, 1003],
+ "sfa":[0.0, 3.01, 30.222, 30.0006789],
+ "soa":[{"in":3},{"in":3,"fl":3.12345}, {"in":3, "fl":30.12345, "nul":"not null"}, {"in":3, "fl":30.6789, "nul":"not null", "bool":true, "str":"here is a string at row 3"}],
+ "oooi":{"oa":{"oab":{"oabc":3}}},
+ "odd": [
+    [[1],[],[3]],
+    [],
+    [[5]]
+ ]
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/test/resources/complex/json/repeated_list.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/complex/json/repeated_list.json b/exec/java-exec/src/test/resources/complex/json/repeated_list.json
index 696d069..b3418f2 100644
--- a/exec/java-exec/src/test/resources/complex/json/repeated_list.json
+++ b/exec/java-exec/src/test/resources/complex/json/repeated_list.json
@@ -4,4 +4,4 @@
     [],
     [[5]]
   ]
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/java-exec/src/test/resources/complex/json/repeated_list_map.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/complex/json/repeated_list_map.json b/exec/java-exec/src/test/resources/complex/json/repeated_list_map.json
index ef6c0ee..cad3fe8 100644
--- a/exec/java-exec/src/test/resources/complex/json/repeated_list_map.json
+++ b/exec/java-exec/src/test/resources/complex/json/repeated_list_map.json
@@ -4,4 +4,4 @@
     [],
     [{"val": [7]}]
   ]
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/ListHolder.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/ListHolder.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/ListHolder.java
new file mode 100644
index 0000000..ebdbae9
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/ListHolder.java
@@ -0,0 +1,37 @@
+/**
+ * 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.expr.holders;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+public class ListHolder implements ValueHolder {
+    public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.LIST);
+    public FieldReader reader;
+    public int isSet;
+
+    public TypeProtos.MajorType getType() {
+        return TYPE;
+    }
+
+    public boolean isSet() {
+        return isSet == 1;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/MapHolder.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/MapHolder.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/MapHolder.java
new file mode 100644
index 0000000..8a38bd4
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/MapHolder.java
@@ -0,0 +1,32 @@
+/**
+ * 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.expr.holders;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+public class MapHolder implements ValueHolder {
+    public static final TypeProtos.MajorType TYPE = Types.required(TypeProtos.MinorType.MAP);
+    public FieldReader reader;
+
+    public TypeProtos.MajorType getType() {
+        return TYPE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedListHolder.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedListHolder.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedListHolder.java
index 09746da..dc857de 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedListHolder.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedListHolder.java
@@ -17,7 +17,23 @@
  */
 package org.apache.drill.exec.expr.holders;
 
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.vector.complex.ListVector;
+
 public final class RepeatedListHolder implements ValueHolder{
-  public int start;
-  public int end;
+
+    public static final TypeProtos.MajorType TYPE = Types.repeated(TypeProtos.MinorType.LIST);
+
+    public TypeProtos.MajorType getType() {return TYPE;}
+
+    /** The first index (inclusive) into the Vector. **/
+    public int start;
+
+    /** The last index (exclusive) into the Vector. **/
+    public int end;
+
+    /** The Vector holding the actual values. **/
+    public ListVector vector;
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f86c4fa8/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedMapHolder.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedMapHolder.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedMapHolder.java
index 247f75e..3db9020 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedMapHolder.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/holders/RepeatedMapHolder.java
@@ -17,7 +17,28 @@
  */
 package org.apache.drill.exec.expr.holders;
 
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+import java.util.LinkedHashSet;
+
 public final class RepeatedMapHolder implements ValueHolder{
-  public int start;
-  public int end;
+
+    public static final TypeProtos.MajorType TYPE = Types.repeated(TypeProtos.MinorType.MAP);
+
+//    public final LinkedHashSet<ValueHolder> children = null;
+
+    public TypeProtos.MajorType getType() {return TYPE;}
+
+    /** The first index (inclusive) into the Vector. **/
+    public int start;
+
+    /** The last index (exclusive) into the Vector. **/
+    public int end;
+
+    /** The Vector holding the actual values. **/
+    public MapVector vector;
+
 }