You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/06/17 17:35:02 UTC

Change in asterixdb[mad-hatter]: [ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates

From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11984 )


Change subject: [ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates
......................................................................

[ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Align type computation for scalar aggregates with
  regular aggregates

Change-Id: Ic587f29840262f052f15e7d257c3d7dda7e40686
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
5 files changed, 93 insertions(+), 61 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/84/11984/1

diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bf2709a..f2a1edf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1771,6 +1771,11 @@
         addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
 
         // Aggregate Functions
+        ScalarVersionOfAggregateResultType scalarNumericSumTypeComputer =
+                new ScalarVersionOfAggregateResultType(NumericSumAggTypeComputer.INSTANCE);
+        ScalarVersionOfAggregateResultType scalarMinMaxTypeComputer =
+                new ScalarVersionOfAggregateResultType(MinMaxAggTypeComputer.INSTANCE);
+
         addPrivateFunction(LISTIFY, OrderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(SCALAR_ARRAYAGG, ScalarArrayAggTypeComputer.INSTANCE, true);
         addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1816,7 +1821,7 @@
 
         // SUM
         addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SUM, scalarNumericSumTypeComputer, true);
         addPrivateFunction(LOCAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1832,8 +1837,8 @@
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MAX, scalarMinMaxTypeComputer, true);
+        addFunction(SCALAR_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_STDDEV_SAMP, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
@@ -1874,7 +1879,7 @@
 
         // SQL SUM
         addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM, scalarNumericSumTypeComputer, true);
         addPrivateFunction(LOCAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1898,8 +1903,8 @@
         addPrivateFunction(GLOBAL_SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_SQL_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MAX, scalarMinMaxTypeComputer, true);
+        addFunction(SCALAR_SQL_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1974,9 +1979,9 @@
         addFunction(SCALAR_SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
 
         addFunction(SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
         addFunction(SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
 
         addFunction(AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1984,14 +1989,14 @@
         addFunction(SCALAR_SQL_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
 
         addFunction(MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
         addFunction(SQL_MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
 
         addFunction(MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
         addFunction(SQL_MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
 
         addFunction(STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
new file mode 100644
index 0000000..8e663a7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AggregateResultTypeComputer extends AbstractResultTypeComputer {
+    @Override
+    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
+        super.checkArgType(funcId, argIndex, type, sourceLoc);
+    }
+
+    @Override
+    protected abstract IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes)
+            throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
index c34b5ed..fc1eee5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.om.typecomputer.impl;
 
 import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -27,7 +26,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 
-public class MinMaxAggTypeComputer extends AbstractResultTypeComputer {
+public class MinMaxAggTypeComputer extends AggregateResultTypeComputer {
 
     public static final MinMaxAggTypeComputer INSTANCE = new MinMaxAggTypeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
index 1c67e56..a4b5e34 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
@@ -18,42 +18,20 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.exceptions.UnsupportedTypeException;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public class NumericSumAggTypeComputer extends AbstractResultTypeComputer {
+public class NumericSumAggTypeComputer extends AggregateResultTypeComputer {
     public static final NumericSumAggTypeComputer INSTANCE = new NumericSumAggTypeComputer();
 
     private NumericSumAggTypeComputer() {
     }
 
     @Override
-    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
-            throws AlgebricksException {
-        ATypeTag tag = type.getTypeTag();
-        switch (tag) {
-            case DOUBLE:
-            case FLOAT:
-            case BIGINT:
-            case INTEGER:
-            case SMALLINT:
-            case TINYINT:
-            case ANY:
-                break;
-            default:
-                throw new UnsupportedTypeException(sourceLoc, funcId, tag);
-        }
-    }
-
-    @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
         ATypeTag tag = strippedInputTypes[0].getTypeTag();
         switch (tag) {
@@ -61,15 +39,12 @@
             case SMALLINT:
             case INTEGER:
             case BIGINT:
-                IAType int64Type = BuiltinType.AINT64;
-                return AUnionType.createNullableType(int64Type, "AggResult");
+                return AUnionType.createNullableType(BuiltinType.AINT64);
             case FLOAT:
             case DOUBLE:
-                IAType doubleType = BuiltinType.ADOUBLE;
-                return AUnionType.createNullableType(doubleType, "AggResult");
+                return AUnionType.createNullableType(BuiltinType.ADOUBLE);
             case ANY:
-                IAType anyType = strippedInputTypes[0];
-                return AUnionType.createNullableType(anyType, "AggResult");
+                return BuiltinType.ANY;
             default:
                 // All other possible cases.
                 return BuiltinType.ANULL;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index 5b90974..fda0285 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -18,9 +18,7 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -32,32 +30,48 @@
 
 public class ScalarVersionOfAggregateResultType extends AbstractResultTypeComputer {
 
-    public static final ScalarVersionOfAggregateResultType INSTANCE = new ScalarVersionOfAggregateResultType();
+    private final AggregateResultTypeComputer aggResultTypeComputer;
 
-    private ScalarVersionOfAggregateResultType() {
+    public ScalarVersionOfAggregateResultType(AggregateResultTypeComputer aggResultTypeComputer) {
+        this.aggResultTypeComputer = aggResultTypeComputer;
     }
 
     @Override
-    public void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType argType, SourceLocation sourceLoc)
             throws AlgebricksException {
-        ATypeTag tag = type.getTypeTag();
-        if (tag != ATypeTag.ANY && tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            throw new TypeMismatchException(sourceLoc, funcId, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
+        if (argIndex == 0) {
+            switch (argType.getTypeTag()) {
+                case ARRAY:
+                case MULTISET:
+                    AbstractCollectionType act = (AbstractCollectionType) argType;
+                    aggResultTypeComputer.checkArgType(funcId, argIndex, act.getItemType(), sourceLoc);
+                    break;
+            }
         }
     }
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        ATypeTag tag = strippedInputTypes[0].getTypeTag();
-        if (tag == ATypeTag.ANY) {
-            return BuiltinType.ANY;
+        IAType argType = strippedInputTypes[0];
+        switch (argType.getTypeTag()) {
+            case ARRAY:
+            case MULTISET:
+                AbstractCollectionType act = (AbstractCollectionType) argType;
+                IAType[] strippedInputTypes2 = strippedInputTypes.clone();
+                strippedInputTypes2[0] = act.getItemType();
+                IAType resultType = aggResultTypeComputer.getResultType(expr, strippedInputTypes2);
+                switch (resultType.getTypeTag()) {
+                    case NULL:
+                    case MISSING:
+                    case ANY:
+                        return resultType;
+                    case UNION:
+                        return AUnionType.createUnknownableType(((AUnionType) resultType).getActualType());
+                    default:
+                        return AUnionType.createUnknownableType(resultType);
+                }
+            default:
+                return BuiltinType.ANY;
         }
-        if (tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            // this condition being true would've thrown an exception above, no?
-            return strippedInputTypes[0];
-        }
-        AbstractCollectionType act = (AbstractCollectionType) strippedInputTypes[0];
-        IAType t = act.getItemType();
-        return AUnionType.createUnknownableType(t);
     }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11984
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: mad-hatter
Gerrit-Change-Id: Ic587f29840262f052f15e7d257c3d7dda7e40686
Gerrit-Change-Number: 11984
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange

Change in asterixdb[mad-hatter]: [ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11984 )


Change subject: [ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates
......................................................................

[ASTERIXDB-2843][COMP] Fix type computer for scalar aggregates

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Align type computation for scalar aggregates with
  regular aggregates

Change-Id: Ic587f29840262f052f15e7d257c3d7dda7e40686
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
5 files changed, 93 insertions(+), 61 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/84/11984/1

diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bf2709a..f2a1edf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1771,6 +1771,11 @@
         addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
 
         // Aggregate Functions
+        ScalarVersionOfAggregateResultType scalarNumericSumTypeComputer =
+                new ScalarVersionOfAggregateResultType(NumericSumAggTypeComputer.INSTANCE);
+        ScalarVersionOfAggregateResultType scalarMinMaxTypeComputer =
+                new ScalarVersionOfAggregateResultType(MinMaxAggTypeComputer.INSTANCE);
+
         addPrivateFunction(LISTIFY, OrderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(SCALAR_ARRAYAGG, ScalarArrayAggTypeComputer.INSTANCE, true);
         addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1816,7 +1821,7 @@
 
         // SUM
         addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SUM, scalarNumericSumTypeComputer, true);
         addPrivateFunction(LOCAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1832,8 +1837,8 @@
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MAX, scalarMinMaxTypeComputer, true);
+        addFunction(SCALAR_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_STDDEV_SAMP, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
@@ -1874,7 +1879,7 @@
 
         // SQL SUM
         addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM, scalarNumericSumTypeComputer, true);
         addPrivateFunction(LOCAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1898,8 +1903,8 @@
         addPrivateFunction(GLOBAL_SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_SQL_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MAX, scalarMinMaxTypeComputer, true);
+        addFunction(SCALAR_SQL_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1974,9 +1979,9 @@
         addFunction(SCALAR_SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
 
         addFunction(SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
         addFunction(SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
 
         addFunction(AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1984,14 +1989,14 @@
         addFunction(SCALAR_SQL_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
 
         addFunction(MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
         addFunction(SQL_MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
 
         addFunction(MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
         addFunction(SQL_MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SCALAR_SQL_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
 
         addFunction(STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
new file mode 100644
index 0000000..8e663a7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AggregateResultTypeComputer extends AbstractResultTypeComputer {
+    @Override
+    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
+        super.checkArgType(funcId, argIndex, type, sourceLoc);
+    }
+
+    @Override
+    protected abstract IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes)
+            throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
index c34b5ed..fc1eee5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.om.typecomputer.impl;
 
 import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -27,7 +26,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 
-public class MinMaxAggTypeComputer extends AbstractResultTypeComputer {
+public class MinMaxAggTypeComputer extends AggregateResultTypeComputer {
 
     public static final MinMaxAggTypeComputer INSTANCE = new MinMaxAggTypeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
index 1c67e56..a4b5e34 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
@@ -18,42 +18,20 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.exceptions.UnsupportedTypeException;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public class NumericSumAggTypeComputer extends AbstractResultTypeComputer {
+public class NumericSumAggTypeComputer extends AggregateResultTypeComputer {
     public static final NumericSumAggTypeComputer INSTANCE = new NumericSumAggTypeComputer();
 
     private NumericSumAggTypeComputer() {
     }
 
     @Override
-    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
-            throws AlgebricksException {
-        ATypeTag tag = type.getTypeTag();
-        switch (tag) {
-            case DOUBLE:
-            case FLOAT:
-            case BIGINT:
-            case INTEGER:
-            case SMALLINT:
-            case TINYINT:
-            case ANY:
-                break;
-            default:
-                throw new UnsupportedTypeException(sourceLoc, funcId, tag);
-        }
-    }
-
-    @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
         ATypeTag tag = strippedInputTypes[0].getTypeTag();
         switch (tag) {
@@ -61,15 +39,12 @@
             case SMALLINT:
             case INTEGER:
             case BIGINT:
-                IAType int64Type = BuiltinType.AINT64;
-                return AUnionType.createNullableType(int64Type, "AggResult");
+                return AUnionType.createNullableType(BuiltinType.AINT64);
             case FLOAT:
             case DOUBLE:
-                IAType doubleType = BuiltinType.ADOUBLE;
-                return AUnionType.createNullableType(doubleType, "AggResult");
+                return AUnionType.createNullableType(BuiltinType.ADOUBLE);
             case ANY:
-                IAType anyType = strippedInputTypes[0];
-                return AUnionType.createNullableType(anyType, "AggResult");
+                return BuiltinType.ANY;
             default:
                 // All other possible cases.
                 return BuiltinType.ANULL;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index 5b90974..fda0285 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -18,9 +18,7 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -32,32 +30,48 @@
 
 public class ScalarVersionOfAggregateResultType extends AbstractResultTypeComputer {
 
-    public static final ScalarVersionOfAggregateResultType INSTANCE = new ScalarVersionOfAggregateResultType();
+    private final AggregateResultTypeComputer aggResultTypeComputer;
 
-    private ScalarVersionOfAggregateResultType() {
+    public ScalarVersionOfAggregateResultType(AggregateResultTypeComputer aggResultTypeComputer) {
+        this.aggResultTypeComputer = aggResultTypeComputer;
     }
 
     @Override
-    public void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+    protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType argType, SourceLocation sourceLoc)
             throws AlgebricksException {
-        ATypeTag tag = type.getTypeTag();
-        if (tag != ATypeTag.ANY && tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            throw new TypeMismatchException(sourceLoc, funcId, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
+        if (argIndex == 0) {
+            switch (argType.getTypeTag()) {
+                case ARRAY:
+                case MULTISET:
+                    AbstractCollectionType act = (AbstractCollectionType) argType;
+                    aggResultTypeComputer.checkArgType(funcId, argIndex, act.getItemType(), sourceLoc);
+                    break;
+            }
         }
     }
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        ATypeTag tag = strippedInputTypes[0].getTypeTag();
-        if (tag == ATypeTag.ANY) {
-            return BuiltinType.ANY;
+        IAType argType = strippedInputTypes[0];
+        switch (argType.getTypeTag()) {
+            case ARRAY:
+            case MULTISET:
+                AbstractCollectionType act = (AbstractCollectionType) argType;
+                IAType[] strippedInputTypes2 = strippedInputTypes.clone();
+                strippedInputTypes2[0] = act.getItemType();
+                IAType resultType = aggResultTypeComputer.getResultType(expr, strippedInputTypes2);
+                switch (resultType.getTypeTag()) {
+                    case NULL:
+                    case MISSING:
+                    case ANY:
+                        return resultType;
+                    case UNION:
+                        return AUnionType.createUnknownableType(((AUnionType) resultType).getActualType());
+                    default:
+                        return AUnionType.createUnknownableType(resultType);
+                }
+            default:
+                return BuiltinType.ANY;
         }
-        if (tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            // this condition being true would've thrown an exception above, no?
-            return strippedInputTypes[0];
-        }
-        AbstractCollectionType act = (AbstractCollectionType) strippedInputTypes[0];
-        IAType t = act.getItemType();
-        return AUnionType.createUnknownableType(t);
     }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11984
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: mad-hatter
Gerrit-Change-Id: Ic587f29840262f052f15e7d257c3d7dda7e40686
Gerrit-Change-Number: 11984
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange