You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2018/11/06 18:41:45 UTC

[1/5] asterixdb git commit: [ASTERIXDB-2460][FUN] Fix sum() overflow bug

Repository: asterixdb
Updated Branches:
  refs/heads/master a993e9bf8 -> 5af85d9ed


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateDescriptor.java
new file mode 100644
index 0000000..9134c81
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalSqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = GlobalSqlSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GLOBAL_SQL_SUM;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new GlobalSqlSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateFunction.java
new file mode 100644
index 0000000..bbff30a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class GlobalSqlSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public GlobalSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finish(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateDescriptor.java
new file mode 100644
index 0000000..930821f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = GlobalSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GLOBAL_SUM;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new GlobalSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateFunction.java
new file mode 100644
index 0000000..9aae327
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class GlobalSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public GlobalSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finish(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        aggType = ATypeTag.NULL;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateDescriptor.java
new file mode 100644
index 0000000..43c1f1a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateSqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.INTERMEDIATE_SQL_SUM;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new IntermediateSqlSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateFunction.java
new file mode 100644
index 0000000..e266516
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class IntermediateSqlSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public IntermediateSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finishPartial(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateDescriptor.java
new file mode 100644
index 0000000..7c76905
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = IntermediateSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.INTERMEDIATE_SUM;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new IntermediateSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateFunction.java
new file mode 100644
index 0000000..2cec29d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class IntermediateSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public IntermediateSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finishPartial(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        aggType = ATypeTag.NULL;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
index 1262399..184f163 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,17 +31,11 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class LocalSqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = BuiltinFunctions.LOCAL_SQL_SUM;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new LocalSqlSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = LocalSqlSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
-        return FID;
+        return BuiltinFunctions.LOCAL_SQL_SUM;
     }
 
     @Override
@@ -53,7 +46,7 @@ public class LocalSqlSumAggregateDescriptor extends AbstractAggregateFunctionDyn
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, true, sourceLoc);
+                return new LocalSqlSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateFunction.java
new file mode 100644
index 0000000..df19968
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateFunction.java
@@ -0,0 +1,84 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class LocalSqlSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public LocalSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finishPartial(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() throws HyracksDataException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
index 0e11541..2fafe32 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,17 +31,11 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class LocalSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = BuiltinFunctions.LOCAL_SUM;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new LocalSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = LocalSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
-        return FID;
+        return BuiltinFunctions.LOCAL_SUM;
     }
 
     @Override
@@ -53,7 +46,7 @@ public class LocalSumAggregateDescriptor extends AbstractAggregateFunctionDynami
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, true, sourceLoc);
+                return new LocalSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateFunction.java
new file mode 100644
index 0000000..0c3a692
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateFunction.java
@@ -0,0 +1,84 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
+
+public class LocalSumAggregateFunction extends AbstractSumAggregateFunction {
+
+    public LocalSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finishPartial(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull() {
+        aggType = ATypeTag.NULL;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() throws HyracksDataException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
index b7fb755..6e73a40 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class SqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SqlSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SqlSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -52,7 +46,7 @@ public class SqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicD
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, false, sourceLoc);
+                return new SqlSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
index 8b4bf7c..0efa4f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
@@ -18,52 +18,64 @@
  */
 package org.apache.asterix.runtime.aggregates.std;
 
-import java.io.IOException;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
 
 public class SqlSumAggregateFunction extends AbstractSumAggregateFunction {
-    private final boolean isLocalAgg;
 
-    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
+    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
             SourceLocation sourceLoc) throws HyracksDataException {
         super(args, context, sourceLoc);
-        this.isLocalAgg = isLocalAgg;
     }
 
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finish(result);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep() {
+        return false;
+    }
+
+    // Handle NULL step
     @Override
     protected void processNull() {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
     }
 
+    // Handle NULL finish
     @Override
-    protected void processSystemNull() throws HyracksDataException {
-        // For global aggregates simply ignore system null here,
-        // but if all input value are system null, then we should return
-        // null in finish().
-        if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
-                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
-        }
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
     }
 
-    @SuppressWarnings("unchecked")
+    // Handle SYSTEM_NULL finish
     @Override
-    protected void finishSystemNull() throws IOException {
-        // Empty stream. For local agg return system null. For global agg return null.
-        if (isLocalAgg) {
-            resultStorage.getDataOutput().writeByte(ATypeTag.SYSTEM_NULL.serialize());
-        } else {
-            serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-            serde.serialize(ANull.NULL, resultStorage.getDataOutput());
-        }
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
index df429b0..1c9623e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class SumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -52,7 +46,7 @@ public class SumAggregateDescriptor extends AbstractAggregateFunctionDynamicDesc
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, false, sourceLoc);
+                return new SumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
index abcac77..51a5e23 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
@@ -18,58 +18,64 @@
  */
 package org.apache.asterix.runtime.aggregates.std;
 
-import java.io.IOException;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.IOException;
 
 public class SumAggregateFunction extends AbstractSumAggregateFunction {
-    private final boolean isLocalAgg;
 
-    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
-            SourceLocation sourceLoc) throws HyracksDataException {
+    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
+            throws HyracksDataException {
         super(args, context, sourceLoc);
-        this.isLocalAgg = isLocalAgg;
     }
 
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        super.step(tuple);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        super.finish(result);
+    }
+
+    // Is skip
     @Override
     protected boolean skipStep() {
-        return (aggType == ATypeTag.NULL);
+        return aggType == ATypeTag.NULL;
     }
 
+    // Handle NULL step
     @Override
     protected void processNull() {
         aggType = ATypeTag.NULL;
     }
 
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
     @Override
-    protected void processSystemNull() throws HyracksDataException {
-        // For global aggregates simply ignore system null here,
-        // but if all input value are system null, then we should return
-        // null in finish().
-        if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
-                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
-        }
+    protected void finishNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
     }
 
-    @SuppressWarnings("unchecked")
+    // Handle SYSTEM_NULL finish
     @Override
-    protected void finishSystemNull() throws IOException {
-        // Empty stream. For local agg return system null. For global agg return null.
-        if (isLocalAgg) {
-            resultStorage.getDataOutput().writeByte(ATypeTag.SYSTEM_NULL.serialize());
-        } else {
-            serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-            serde.serialize(ANull.NULL, resultStorage.getDataOutput());
-        }
+    protected void finishSystemNull(IPointable result) throws IOException {
+        resultStorage.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        result.set(resultStorage);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 1b0c7b6..dd1d862 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -55,20 +55,24 @@ import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobal
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlStddevPopAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlVarPopAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlStddevPopAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlVarPopAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalAvgAggregateDescriptor;
@@ -98,6 +102,10 @@ import org.apache.asterix.runtime.aggregates.serializable.std.SerializableVarPop
 import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.RangeMapAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
@@ -509,6 +517,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(GlobalAvgAggregateDescriptor.FACTORY);
         fc.add(SumAggregateDescriptor.FACTORY);
         fc.add(LocalSumAggregateDescriptor.FACTORY);
+        fc.add(IntermediateSumAggregateDescriptor.FACTORY);
+        fc.add(GlobalSumAggregateDescriptor.FACTORY);
         fc.add(MaxAggregateDescriptor.FACTORY);
         fc.add(LocalMaxAggregateDescriptor.FACTORY);
         fc.add(MinAggregateDescriptor.FACTORY);
@@ -542,6 +552,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(SerializableGlobalAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableSumAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalSumAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateSumAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalSumAggregateDescriptor.FACTORY);
         fc.add(SerializableStddevAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalStddevAggregateDescriptor.FACTORY);
         fc.add(SerializableIntermediateStddevAggregateDescriptor.FACTORY);
@@ -580,6 +592,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(GlobalSqlAvgAggregateDescriptor.FACTORY);
         fc.add(SqlSumAggregateDescriptor.FACTORY);
         fc.add(LocalSqlSumAggregateDescriptor.FACTORY);
+        fc.add(IntermediateSqlSumAggregateDescriptor.FACTORY);
+        fc.add(GlobalSqlSumAggregateDescriptor.FACTORY);
         fc.add(SqlMaxAggregateDescriptor.FACTORY);
         fc.add(LocalSqlMaxAggregateDescriptor.FACTORY);
         fc.add(SqlMinAggregateDescriptor.FACTORY);
@@ -609,6 +623,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(SerializableGlobalSqlAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableSqlSumAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalSqlSumAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateSqlSumAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalSqlSumAggregateDescriptor.FACTORY);
         fc.add(SerializableSqlStddevAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalSqlStddevAggregateDescriptor.FACTORY);
         fc.add(SerializableIntermediateSqlStddevAggregateDescriptor.FACTORY);


[5/5] asterixdb git commit: [ASTERIXDB-2460][FUN] Fix sum() overflow bug

Posted by dl...@apache.org.
[ASTERIXDB-2460][FUN] Fix sum() overflow bug

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

Details:
- Fixed the sum() overflow problem. Now the sum() function will
return either bigint or double based on the input types.

- Added sum() aggregate local, intermediate and global classes.
Currently, it is using the same class for the three steps and
checking the type using an if statement. Although the three
steps all have a very similar behavior, having them in split
classes makes it cleaner and easier to read and maintain in the
future.

- Added sum() aggregate local, intermediate and global for serialized
classes.

- Added sum() and serial sum() test cases and modified test cases
to cause the reported overflow issue.

- Added sum() and serial sum() test cases that cause a
Long.MAX_VALUE type overflow.

- Valid data type check is now part of the sum calculation to
avoid redundant steps.

Change-Id: I987417770b3bfbda6af29a27acc8c96dc8a99eb8
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3012
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>


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

Branch: refs/heads/master
Commit: 5af85d9ed157ad6f942712ea5e6108ea3f1544e0
Parents: a993e9b
Author: Hussain Towaileb <Hu...@Gmail.com>
Authored: Tue Nov 6 13:12:40 2018 +0300
Committer: Dmitry Lychagin <dm...@couchbase.com>
Committed: Tue Nov 6 10:41:23 2018 -0800

----------------------------------------------------------------------
 .../serial_sum_double.1.ddl.sqlpp               |  33 +++
 .../serial_sum_double.2.update.sqlpp            |  31 +++
 .../serial_sum_double.3.query.sqlpp             |  25 ++
 .../serial_sum_double_null.1.ddl.sqlpp          |  33 +++
 .../serial_sum_double_null.2.update.sqlpp       |  31 +++
 .../serial_sum_double_null.3.query.sqlpp        |  25 ++
 .../serial_sum_empty.1.ddl.sqlpp                |  33 +++
 .../serial_sum_empty.2.update.sqlpp             |  18 ++
 .../serial_sum_empty.3.query.sqlpp              |  25 ++
 .../serial_sum_float.1.ddl.sqlpp                |  33 +++
 .../serial_sum_float.2.update.sqlpp             |  31 +++
 .../serial_sum_float.3.query.sqlpp              |  25 ++
 .../serial_sum_float_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_float_null.2.update.sqlpp        |  31 +++
 .../serial_sum_float_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int16.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int16.2.update.sqlpp             |  31 +++
 .../serial_sum_int16.3.query.sqlpp              |  25 ++
 .../serial_sum_int16_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int16_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int16_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int32.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int32.2.update.sqlpp             |  31 +++
 .../serial_sum_int32.3.query.sqlpp              |  25 ++
 .../serial_sum_int32_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int32_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int32_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int64.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int64.2.update.sqlpp             |  31 +++
 .../serial_sum_int64.3.query.sqlpp              |  25 ++
 .../serial_sum_int64_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int64_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int64_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int64_overflow.1.ddl.sqlpp       |  33 +++
 .../serial_sum_int64_overflow.2.update.sqlpp    |  33 +++
 .../serial_sum_int64_overflow.3.query.sqlpp     |  25 ++
 .../serial_sum_int8/serial_sum_int8.1.ddl.sqlpp |  33 +++
 .../serial_sum_int8.2.update.sqlpp              |  31 +++
 .../serial_sum_int8.3.query.sqlpp               |  25 ++
 .../serial_sum_int8_null.1.ddl.sqlpp            |  33 +++
 .../serial_sum_int8_null.2.update.sqlpp         |  31 +++
 .../serial_sum_int8_null.3.query.sqlpp          |  25 ++
 .../serial_sum_mixed.1.ddl.sqlpp                |  33 +++
 .../serial_sum_mixed.2.update.sqlpp             |  29 +++
 .../serial_sum_mixed.3.query.sqlpp              |  31 +++
 .../sum_int64_overflow.1.ddl.sqlpp              |  22 ++
 .../sum_int64_overflow.2.update.sqlpp           |  19 ++
 .../sum_int64_overflow.3.query.sqlpp            |  26 ++
 .../sum_int8/sum_int8.3.query.sqlpp             |   2 +-
 .../serial_sum_double.1.ddl.sqlpp               |  33 +++
 .../serial_sum_double.2.update.sqlpp            |  31 +++
 .../serial_sum_double.3.query.sqlpp             |  25 ++
 .../serial_sum_double_null.1.ddl.sqlpp          |  33 +++
 .../serial_sum_double_null.2.update.sqlpp       |  31 +++
 .../serial_sum_double_null.3.query.sqlpp        |  25 ++
 .../serial_sum_empty.1.ddl.sqlpp                |  33 +++
 .../serial_sum_empty.2.update.sqlpp             |  18 ++
 .../serial_sum_empty.3.query.sqlpp              |  25 ++
 .../serial_sum_float.1.ddl.sqlpp                |  33 +++
 .../serial_sum_float.2.update.sqlpp             |  31 +++
 .../serial_sum_float.3.query.sqlpp              |  25 ++
 .../serial_sum_float_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_float_null.2.update.sqlpp        |  31 +++
 .../serial_sum_float_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int16.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int16.2.update.sqlpp             |  31 +++
 .../serial_sum_int16.3.query.sqlpp              |  25 ++
 .../serial_sum_int16_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int16_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int16_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int32.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int32.2.update.sqlpp             |  31 +++
 .../serial_sum_int32.3.query.sqlpp              |  25 ++
 .../serial_sum_int32_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int32_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int32_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int64.1.ddl.sqlpp                |  33 +++
 .../serial_sum_int64.2.update.sqlpp             |  31 +++
 .../serial_sum_int64.3.query.sqlpp              |  25 ++
 .../serial_sum_int64_null.1.ddl.sqlpp           |  33 +++
 .../serial_sum_int64_null.2.update.sqlpp        |  31 +++
 .../serial_sum_int64_null.3.query.sqlpp         |  25 ++
 .../serial_sum_int64_overflow.1.ddl.sqlpp       |  33 +++
 .../serial_sum_int64_overflow.2.update.sqlpp    |  33 +++
 .../serial_sum_int64_overflow.3.query.sqlpp     |  25 ++
 .../serial_sum_int8/serial_sum_int8.1.ddl.sqlpp |  33 +++
 .../serial_sum_int8.2.update.sqlpp              |  31 +++
 .../serial_sum_int8.3.query.sqlpp               |  25 ++
 .../serial_sum_int8_null.1.ddl.sqlpp            |  33 +++
 .../serial_sum_int8_null.2.update.sqlpp         |  31 +++
 .../serial_sum_int8_null.3.query.sqlpp          |  25 ++
 .../serial_sum_mixed.1.ddl.sqlpp                |  33 +++
 .../serial_sum_mixed.2.update.sqlpp             |  29 +++
 .../serial_sum_mixed.3.query.sqlpp              |  30 +++
 .../sum_int64_overflow.1.ddl.sqlpp              |  22 ++
 .../sum_int64_overflow.2.update.sqlpp           |  19 ++
 .../sum_int64_overflow.3.query.sqlpp            |  26 ++
 .../aggregate/sum_int8/sum_int8.3.query.sqlpp   |   2 +-
 .../serial_sum_double/serial_sum_double.1.adm   |   1 +
 .../serial_sum_double_null.1.adm                |   1 +
 .../serial_sum_empty/serial_sum_empty.1.adm     |   0
 .../serial_sum_float/serial_sum_float.1.adm     |   1 +
 .../serial_sum_float_null.1.adm                 |   1 +
 .../serial_sum_int16/serial_sum_int16.1.adm     |   1 +
 .../serial_sum_int16_null.1.adm                 |   1 +
 .../serial_sum_int32/serial_sum_int32.1.adm     |   1 +
 .../serial_sum_int32_null.1.adm                 |   1 +
 .../serial_sum_int64/serial_sum_int64.1.adm     |   1 +
 .../serial_sum_int64_null.1.adm                 |   1 +
 .../serial_sum_int8/serial_sum_int8.1.adm       |   1 +
 .../serial_sum_int8_null.1.adm                  |   1 +
 .../serial_sum_mixed/serial_sum_mixed.1.adm     |   1 +
 .../aggregate-sql/sum_int8/sum_int8.1.adm       |   2 +-
 .../serial_sum_double/serial_sum_double.1.adm   |   1 +
 .../serial_sum_double_null.1.adm                |   1 +
 .../serial_sum_empty/serial_sum_empty.1.adm     |   0
 .../serial_sum_float/serial_sum_float.1.adm     |   1 +
 .../serial_sum_float_null.1.adm                 |   1 +
 .../serial_sum_int16/serial_sum_int16.1.adm     |   1 +
 .../serial_sum_int16_null.1.adm                 |   1 +
 .../serial_sum_int32/serial_sum_int32.1.adm     |   1 +
 .../serial_sum_int32_null.1.adm                 |   1 +
 .../serial_sum_int64/serial_sum_int64.1.adm     |   1 +
 .../serial_sum_int64_null.1.adm                 |   1 +
 .../serial_sum_int8/serial_sum_int8.1.adm       |   1 +
 .../serial_sum_int8_null.1.adm                  |   1 +
 .../serial_sum_mixed/serial_sum_mixed.1.adm     |   1 +
 .../results/aggregate/sum_int8/sum_int8.1.adm   |   2 +-
 .../explain_field_access.1.adm                  |   2 +-
 .../explain_field_access_closed.1.adm           |   2 +-
 .../aggregate-sql/sum_int8/sum_int8.3.ast       |   6 +-
 .../aggregate/sum_int8/sum_int8.3.ast           |   6 +-
 .../resources/runtimets/testsuite_sqlpp.xml     | 215 ++++++++++++++--
 .../asterix/om/functions/BuiltinFunctions.java  | 108 +++++---
 .../impl/NumericAggTypeComputer.java            |  74 ------
 .../impl/NumericSumAggTypeComputer.java         |  78 ++++++
 .../scalar/ScalarSqlSumAggregateDescriptor.java |  12 +-
 .../scalar/ScalarSumAggregateDescriptor.java    |  12 +-
 ...bstractSerializableSumAggregateFunction.java | 258 +++++++++++--------
 ...alizableGlobalSqlSumAggregateDescriptor.java |  55 ++++
 ...rializableGlobalSqlSumAggregateFunction.java |  79 ++++++
 ...erializableGlobalSumAggregateDescriptor.java |  54 ++++
 .../SerializableGlobalSumAggregateFunction.java |  81 ++++++
 ...leIntermediateSqlSumAggregateDescriptor.java |  55 ++++
 ...ableIntermediateSqlSumAggregateFunction.java |  79 ++++++
 ...zableIntermediateSumAggregateDescriptor.java |  55 ++++
 ...lizableIntermediateSumAggregateFunction.java |  81 ++++++
 ...ializableLocalSqlSumAggregateDescriptor.java |  10 +-
 ...erializableLocalSqlSumAggregateFunction.java |  82 ++++++
 ...SerializableLocalSumAggregateDescriptor.java |  12 +-
 .../SerializableLocalSumAggregateFunction.java  |  84 ++++++
 .../SerializableSqlSumAggregateDescriptor.java  |  10 +-
 .../SerializableSqlSumAggregateFunction.java    |  66 ++---
 .../std/SerializableSumAggregateDescriptor.java |  10 +-
 .../std/SerializableSumAggregateFunction.java   |  64 ++---
 .../std/AbstractSumAggregateFunction.java       | 237 +++++++++--------
 .../std/GlobalSqlSumAggregateDescriptor.java    |  53 ++++
 .../std/GlobalSqlSumAggregateFunction.java      |  81 ++++++
 .../std/GlobalSumAggregateDescriptor.java       |  53 ++++
 .../std/GlobalSumAggregateFunction.java         |  81 ++++++
 .../IntermediateSqlSumAggregateDescriptor.java  |  53 ++++
 .../IntermediateSqlSumAggregateFunction.java    |  81 ++++++
 .../std/IntermediateSumAggregateDescriptor.java |  53 ++++
 .../std/IntermediateSumAggregateFunction.java   |  81 ++++++
 .../std/LocalSqlSumAggregateDescriptor.java     |  13 +-
 .../std/LocalSqlSumAggregateFunction.java       |  84 ++++++
 .../std/LocalSumAggregateDescriptor.java        |  13 +-
 .../std/LocalSumAggregateFunction.java          |  84 ++++++
 .../std/SqlSumAggregateDescriptor.java          |  10 +-
 .../aggregates/std/SqlSumAggregateFunction.java |  66 +++--
 .../aggregates/std/SumAggregateDescriptor.java  |  10 +-
 .../aggregates/std/SumAggregateFunction.java    |  66 ++---
 .../runtime/functions/FunctionCollection.java   |  16 ++
 173 files changed, 5040 insertions(+), 568 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double/serial_sum_double.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.2.update.sqlpp
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_empty/serial_sum_empty.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : float
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float/serial_sum_float.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : float
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : smallint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16/serial_sum_int16.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : smallint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : integer
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32/serial_sum_int32.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : integer
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64/serial_sum_int64.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
new file mode 100644
index 0000000..5efbe51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
+insert into Test
+select element {'id':5,'gid':1,'val':bigint('9223372036854775807'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : tinyint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.3.query.sqlpp
new file mode 100644
index 0000000..4259ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8/serial_sum_int8.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.val) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : tinyint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
new file mode 100644
index 0000000..5ed57e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, sum(t.valplus) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid;


[3/5] asterixdb git commit: [ASTERIXDB-2460][FUN] Fix sum() overflow bug

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
new file mode 100644
index 0000000..1b4eac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':float('2.0')};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32),'valplus':float('1.0')};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':'hello world'};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double('3.0')};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
new file mode 100644
index 0000000..96551e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+ /*
+ * Description  : Run serial-sum over an ordered list with mixed types
+ * Expected Res : Failure
+ * Date         : March 5th 2018
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
new file mode 100644
index 0000000..21479a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
new file mode 100644
index 0000000..2520b7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+select element test.strict_sum((
+    select element x
+    from  [test.bigint('1'),test.bigint('2'),test.bigint('3'),test.bigint('9223372036854775807')] as x
+));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
index 595ecca..d592b74 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
@@ -22,5 +22,5 @@ use test;
 
 select element test.strict_sum((
     select element x
-    from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
+    from  [test.tinyint('100'),test.tinyint('100'),test.tinyint('100')] as x
 ));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double/serial_sum_double.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double/serial_sum_double.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double/serial_sum_double.1.adm
new file mode 100644
index 0000000..26d8cc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double/serial_sum_double.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 24.6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.adm
new file mode 100644
index 0000000..56b8cdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_double_null/serial_sum_double_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 38473827485890992.0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_empty/serial_sum_empty.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_empty/serial_sum_empty.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_empty/serial_sum_empty.1.adm
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float/serial_sum_float.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float/serial_sum_float.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float/serial_sum_float.1.adm
new file mode 100644
index 0000000..26d8cc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float/serial_sum_float.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 24.6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.adm
new file mode 100644
index 0000000..d28ddd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_float_null/serial_sum_float_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 6.0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16/serial_sum_int16.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16/serial_sum_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16/serial_sum_int16.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16/serial_sum_int16.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.adm
new file mode 100644
index 0000000..d41f389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int16_null/serial_sum_int16_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32/serial_sum_int32.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32/serial_sum_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32/serial_sum_int32.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32/serial_sum_int32.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.adm
new file mode 100644
index 0000000..d41f389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int32_null/serial_sum_int32_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64/serial_sum_int64.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64/serial_sum_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64/serial_sum_int64.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64/serial_sum_int64.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.adm
new file mode 100644
index 0000000..d41f389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int64_null/serial_sum_int64_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8/serial_sum_int8.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8/serial_sum_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8/serial_sum_int8.1.adm
new file mode 100644
index 0000000..663d0a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8/serial_sum_int8.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 370 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.adm
new file mode 100644
index 0000000..7f976d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_int8_null/serial_sum_int8_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 300 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/sum_int8/sum_int8.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/sum_int8/sum_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/sum_int8/sum_int8.1.adm
index 1e8b314..697cb3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/sum_int8/sum_int8.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/sum_int8/sum_int8.1.adm
@@ -1 +1 @@
-6
+300

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double/serial_sum_double.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double/serial_sum_double.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double/serial_sum_double.1.adm
new file mode 100644
index 0000000..26d8cc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double/serial_sum_double.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 24.6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double_null/serial_sum_double_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double_null/serial_sum_double_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double_null/serial_sum_double_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_double_null/serial_sum_double_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_empty/serial_sum_empty.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_empty/serial_sum_empty.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_empty/serial_sum_empty.1.adm
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float/serial_sum_float.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float/serial_sum_float.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float/serial_sum_float.1.adm
new file mode 100644
index 0000000..26d8cc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float/serial_sum_float.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 24.6 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float_null/serial_sum_float_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float_null/serial_sum_float_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float_null/serial_sum_float_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_float_null/serial_sum_float_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16/serial_sum_int16.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16/serial_sum_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16/serial_sum_int16.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16/serial_sum_int16.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32/serial_sum_int32.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32/serial_sum_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32/serial_sum_int32.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32/serial_sum_int32.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64/serial_sum_int64.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64/serial_sum_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64/serial_sum_int64.1.adm
new file mode 100644
index 0000000..60e711a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64/serial_sum_int64.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 23 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8/serial_sum_int8.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8/serial_sum_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8/serial_sum_int8.1.adm
new file mode 100644
index 0000000..663d0a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8/serial_sum_int8.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": 370 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_mixed/serial_sum_mixed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_mixed/serial_sum_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_mixed/serial_sum_mixed.1.adm
new file mode 100644
index 0000000..eb5581f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_sum_mixed/serial_sum_mixed.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "sum": null }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/sum_int8/sum_int8.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/sum_int8/sum_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/sum_int8/sum_int8.1.adm
index 1e8b314..697cb3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/sum_int8/sum_int8.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/sum_int8/sum_int8.1.adm
@@ -1 +1 @@
-6
+300

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
index 384e8f8..4e079d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
@@ -9,7 +9,7 @@ distribute result [$$40]
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           group by ([$#1 := $$48]) decor ([]) {
-                    aggregate [$$43] <- [agg-sql-sum($$47)]
+                    aggregate [$$43] <- [agg-global-sql-sum($$47)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
index b82dda7..2477020 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
@@ -9,7 +9,7 @@ distribute result [$$39]
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           group by ([$#1 := $$46]) decor ([]) {
-                    aggregate [$$42] <- [agg-sql-sum($$45)]
+                    aggregate [$$42] <- [agg-global-sql-sum($$45)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/sum_int8/sum_int8.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/sum_int8/sum_int8.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/sum_int8/sum_int8.3.ast
index 350605d..cd32941 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/sum_int8/sum_int8.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/sum_int8/sum_int8.3.ast
@@ -8,13 +8,13 @@ FunctionCall asterix.sql-sum@1[
     ]
     FROM [      OrderedListConstructor [
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [1]
+          LiteralExpr [STRING] [100]
         ]
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [2]
+          LiteralExpr [STRING] [100]
         ]
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [3]
+          LiteralExpr [STRING] [100]
         ]
       ]
       AS Variable [ Name=$x ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/sum_int8/sum_int8.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/sum_int8/sum_int8.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/sum_int8/sum_int8.3.ast
index 11eb977..8cd57a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/sum_int8/sum_int8.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/sum_int8/sum_int8.3.ast
@@ -8,13 +8,13 @@ FunctionCall asterix.sum@1[
     ]
     FROM [      OrderedListConstructor [
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [1]
+          LiteralExpr [STRING] [100]
         ]
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [2]
+          LiteralExpr [STRING] [100]
         ]
         FunctionCall test.int8@1[
-          LiteralExpr [STRING] [3]
+          LiteralExpr [STRING] [100]
         ]
       ]
       AS Variable [ Name=$x ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 18455df..619d12c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -194,12 +194,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="sum_mixed">
-        <output-dir compare="Text">sum_mixed</output-dir>
-        <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
       <compilation-unit name="min_mixed">
         <output-dir compare="Text">min_mixed</output-dir>
         <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
@@ -230,6 +224,18 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
+      <compilation-unit name="sum_mixed">
+        <output-dir compare="Text">sum_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-sum cannot process item type string in an input array</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_mixed">
+        <output-dir compare="Text">serial_sum_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-sum cannot process item type string in an input array</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="var_mixed">
         <output-dir compare="Text">var_mixed</output-dir>
         <expected-error>Type incompatibility: function agg-var gets incompatible input values: string and float</expected-error>
@@ -590,28 +596,28 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="scalar_var">
-        <output-dir compare="Text">scalar_var</output-dir>
+      <compilation-unit name="scalar_sum_empty">
+        <output-dir compare="Text">scalar_sum_empty</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="scalar_var_empty">
-        <output-dir compare="Text">scalar_var_empty</output-dir>
+      <compilation-unit name="scalar_sum_null">
+        <output-dir compare="Text">scalar_sum_null</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="scalar_var_null">
-        <output-dir compare="Text">scalar_var_null</output-dir>
+      <compilation-unit name="scalar_var">
+        <output-dir compare="Text">scalar_var</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="scalar_sum_empty">
-        <output-dir compare="Text">scalar_sum_empty</output-dir>
+      <compilation-unit name="scalar_var_empty">
+        <output-dir compare="Text">scalar_var_empty</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
-      <compilation-unit name="scalar_sum_null">
-        <output-dir compare="Text">scalar_sum_null</output-dir>
+      <compilation-unit name="scalar_var_null">
+        <output-dir compare="Text">scalar_var_null</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
@@ -680,6 +686,77 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_double">
+        <output-dir compare="Text">serial_sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_double_null">
+        <output-dir compare="Text">serial_sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_empty">
+        <output-dir compare="Text">serial_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_float">
+        <output-dir compare="Text">serial_sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_float_null">
+        <output-dir compare="Text">serial_sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int16">
+        <output-dir compare="Text">serial_sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int16_null">
+        <output-dir compare="Text">serial_sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int32">
+        <output-dir compare="Text">serial_sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int32_null">
+        <output-dir compare="Text">serial_sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int64">
+        <output-dir compare="Text">serial_sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+    <compilation-unit name="serial_sum_int64_null">
+      <output-dir compare="Text">serial_sum_int64_null</output-dir>
+    </compilation-unit>
+  </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int64_overflow">
+        <output-dir compare="Text">serial_sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int8">
+        <output-dir compare="Text">serial_sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_sum_int8_null">
+        <output-dir compare="Text">serial_sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="serial_var_double">
         <output-dir compare="Text">serial_var_double</output-dir>
       </compilation-unit>
@@ -880,6 +957,17 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
+      <compilation-unit name="sum_int64_null">
+        <output-dir compare="Text">sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum_int64_overflow">
+        <output-dir compare="Text">sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="sum_int8">
         <output-dir compare="Text">sum_int8</output-dir>
       </compilation-unit>
@@ -1013,12 +1101,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
-      <compilation-unit name="sum_mixed">
-        <output-dir compare="Text">sum_mixed</output-dir>
-        <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate-sql">
       <compilation-unit name="min_mixed">
         <output-dir compare="Text">min_mixed</output-dir>
         <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
@@ -1049,6 +1131,18 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum_mixed">
+        <output-dir compare="Text">sum_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-sum cannot process item type string in an input array</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_mixed">
+        <output-dir compare="Text">serial_sum_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-sum cannot process item type string in an input array</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="serial_var_mixed">
         <output-dir compare="Text">serial_var_mixed</output-dir>
         <expected-error>Invalid item type: function agg-var cannot process item type string in an input array (or multiset)</expected-error>
@@ -1428,6 +1522,77 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_double">
+        <output-dir compare="Text">serial_sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_double_null">
+        <output-dir compare="Text">serial_sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_empty">
+        <output-dir compare="Text">serial_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_float">
+        <output-dir compare="Text">serial_sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_float_null">
+        <output-dir compare="Text">serial_sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int16">
+        <output-dir compare="Text">serial_sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int16_null">
+        <output-dir compare="Text">serial_sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int32">
+        <output-dir compare="Text">serial_sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int32_null">
+        <output-dir compare="Text">serial_sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int64">
+        <output-dir compare="Text">serial_sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+    <compilation-unit name="serial_sum_int64_null">
+      <output-dir compare="Text">serial_sum_int64_null</output-dir>
+    </compilation-unit>
+  </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int64_overflow">
+        <output-dir compare="Text">serial_sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int8">
+        <output-dir compare="Text">serial_sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_sum_int8_null">
+        <output-dir compare="Text">serial_sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="serial_var_double">
         <output-dir compare="Text">serial_var_double</output-dir>
       </compilation-unit>
@@ -1628,6 +1793,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum_int64_overflow">
+        <output-dir compare="Text">sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="sum_int8">
         <output-dir compare="Text">sum_int8</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
----------------------------------------------------------------------
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 ccb124a..9a52f55 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
@@ -89,7 +89,7 @@ import org.apache.asterix.om.typecomputer.impl.NotUnknownTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NullIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericAddSubMulDivTypeComputer;
-import org.apache.asterix.om.typecomputer.impl.NumericAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NumericSumAggTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericDivideTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericDoubleOutputFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericInt8OutputFunctionTypeComputer;
@@ -460,6 +460,10 @@ public class BuiltinFunctions {
     public static final FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
     public static final FunctionIdentifier LOCAL_SUM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sum", 1);
+    public static final FunctionIdentifier INTERMEDIATE_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-sum", 1);
+    public static final FunctionIdentifier GLOBAL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sum", 1);
     public static final FunctionIdentifier MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1);
     public static final FunctionIdentifier LOCAL_MAX =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-max", 1);
@@ -557,6 +561,10 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum-serial", 1);
     public static final FunctionIdentifier SERIAL_LOCAL_SUM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "local-sum-serial", 1);
+    public static final FunctionIdentifier SERIAL_INTERMEDIATE_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "intermediate-sum-serial", 1);
+    public static final FunctionIdentifier SERIAL_GLOBAL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "global-sum-serial", 1);
     public static final FunctionIdentifier SERIAL_GLOBAL_AVG =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "global-avg-serial", 1);
     public static final FunctionIdentifier SERIAL_LOCAL_AVG =
@@ -645,6 +653,10 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-sum", 1);
     public static final FunctionIdentifier LOCAL_SQL_SUM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-sum", 1);
+    public static final FunctionIdentifier INTERMEDIATE_SQL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-sql-sum", 1);
+    public static final FunctionIdentifier GLOBAL_SQL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-sum", 1);
     public static final FunctionIdentifier SQL_MAX =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-max", 1);
     public static final FunctionIdentifier LOCAL_SQL_MAX =
@@ -738,6 +750,10 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-sum-serial", 1);
     public static final FunctionIdentifier SERIAL_LOCAL_SQL_SUM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "local-sql-sum-serial", 1);
+    public static final FunctionIdentifier SERIAL_INTERMEDIATE_SQL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "intermediate-sql-sum-serial", 1);
+    public static final FunctionIdentifier SERIAL_GLOBAL_SQL_SUM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "global-sql-sum-serial", 1);
     public static final FunctionIdentifier SERIAL_GLOBAL_SQL_AVG =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "global-sql-avg-serial", 1);
     public static final FunctionIdentifier SERIAL_INTERMEDIATE_SQL_AVG =
@@ -1565,8 +1581,6 @@ public class BuiltinFunctions {
         addFunction(COUNT, AInt64TypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(AVG, NullableDoubleTypeComputer.INSTANCE, true);
-        addFunction(SUM, NumericAggTypeComputer.INSTANCE, true);
-        addPrivateFunction(LOCAL_SUM, NumericAggTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_FIRST_ELEMENT, CollectionMemberResultType.INSTANCE, true);
         addPrivateFunction(FIRST_ELEMENT, PropagateTypeComputer.INSTANCE, true);
@@ -1586,20 +1600,28 @@ public class BuiltinFunctions {
         addFunction(VAR_POP, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_VAR_POP, NullableDoubleTypeComputer.INSTANCE, true);
 
+        // SUM
+        addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addPrivateFunction(LOCAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERMEDIATE_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(GLOBAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_LOCAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_INTERMEDIATE_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_GLOBAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+
         addPrivateFunction(SERIAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_GLOBAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
-        addPrivateFunction(SERIAL_SQL_SUM, NumericAggTypeComputer.INSTANCE, true);
-        addPrivateFunction(SERIAL_LOCAL_SQL_SUM, NumericAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_LOCAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
         addFunction(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_GLOBAL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1634,6 +1656,17 @@ public class BuiltinFunctions {
         addPrivateFunction(SERIAL_LOCAL_SQL_VAR_POP, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_VAR_POP, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
 
+        // SQL SUM
+        addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addPrivateFunction(LOCAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERMEDIATE_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(GLOBAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_LOCAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_INTERMEDIATE_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_GLOBAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
+
         addFunction(SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
@@ -1643,15 +1676,12 @@ public class BuiltinFunctions {
         addPrivateFunction(LOCAL_SQL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
         addFunction(SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
-        addFunction(SQL_SUM, NumericAggTypeComputer.INSTANCE, true);
-        addPrivateFunction(LOCAL_SQL_SUM, NumericAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_GLOBAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SCALAR_LOCAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
         addFunction(SCALAR_SQL_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SQL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_SQL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1687,8 +1717,6 @@ public class BuiltinFunctions {
         addPrivateFunction(SERIAL_GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
-        addPrivateFunction(SERIAL_SUM, NumericAggTypeComputer.INSTANCE, true);
-        addPrivateFunction(SERIAL_LOCAL_SUM, NumericAggTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_GLOBAL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_STDDEV, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
@@ -1713,10 +1741,10 @@ public class BuiltinFunctions {
         addFunction(SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
 
-        addFunction(SUM_DISTINCT, NumericAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
-        addFunction(SQL_SUM_DISTINCT, NumericAggTypeComputer.INSTANCE, true);
-        addFunction(SCALAR_SQL_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+        addFunction(SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
+        addFunction(SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
 
         addFunction(AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
@@ -2251,27 +2279,29 @@ public class BuiltinFunctions {
         addScalarAgg(MIN_DISTINCT, SCALAR_MIN_DISTINCT);
 
         // SUM
-
         addAgg(SUM);
         addAgg(LOCAL_SUM);
+        addAgg(GLOBAL_SUM);
         addLocalAgg(SUM, LOCAL_SUM);
-        addIntermediateAgg(LOCAL_SUM, SUM);
-        addIntermediateAgg(SUM, SUM);
-        addGlobalAgg(SUM, SUM);
-
+        addIntermediateAgg(SUM, INTERMEDIATE_SUM);
+        addIntermediateAgg(LOCAL_SUM, INTERMEDIATE_SUM);
+        addIntermediateAgg(GLOBAL_SUM, INTERMEDIATE_SUM);
+        addGlobalAgg(SUM, GLOBAL_SUM);
         addScalarAgg(SUM, SCALAR_SUM);
 
-        addSerialAgg(SUM, SERIAL_SUM);
-        addSerialAgg(LOCAL_SUM, SERIAL_LOCAL_SUM);
         addAgg(SERIAL_SUM);
         addAgg(SERIAL_LOCAL_SUM);
+        addAgg(SERIAL_GLOBAL_SUM);
+        addSerialAgg(SUM, SERIAL_SUM);
+        addSerialAgg(LOCAL_SUM, SERIAL_LOCAL_SUM);
+        addSerialAgg(GLOBAL_SUM, SERIAL_GLOBAL_SUM);
         addLocalAgg(SERIAL_SUM, SERIAL_LOCAL_SUM);
-        addIntermediateAgg(SERIAL_SUM, SERIAL_SUM);
-        addIntermediateAgg(SERIAL_LOCAL_SUM, SERIAL_SUM);
-        addGlobalAgg(SERIAL_SUM, SERIAL_SUM);
-
-        // SUM DISTINCT
+        addIntermediateAgg(SERIAL_SUM, SERIAL_INTERMEDIATE_SUM);
+        addIntermediateAgg(SERIAL_LOCAL_SUM, SERIAL_INTERMEDIATE_SUM);
+        addIntermediateAgg(SERIAL_GLOBAL_SUM, SERIAL_INTERMEDIATE_SUM);
+        addGlobalAgg(SERIAL_SUM, SERIAL_GLOBAL_SUM);
 
+        // SUM Distinct
         addDistinctAgg(SUM_DISTINCT, SCALAR_SUM);
         addScalarAgg(SUM_DISTINCT, SCALAR_SUM_DISTINCT);
 
@@ -2493,27 +2523,29 @@ public class BuiltinFunctions {
         addScalarAgg(SQL_MIN_DISTINCT, SCALAR_SQL_MIN_DISTINCT);
 
         // SQL SUM
-
         addAgg(SQL_SUM);
         addAgg(LOCAL_SQL_SUM);
+        addAgg(GLOBAL_SQL_SUM);
         addLocalAgg(SQL_SUM, LOCAL_SQL_SUM);
-        addIntermediateAgg(LOCAL_SQL_SUM, SQL_SUM);
-        addIntermediateAgg(SQL_SUM, SQL_SUM);
-        addGlobalAgg(SQL_SUM, SQL_SUM);
-
+        addIntermediateAgg(SQL_SUM, INTERMEDIATE_SQL_SUM);
+        addIntermediateAgg(LOCAL_SQL_SUM, INTERMEDIATE_SQL_SUM);
+        addIntermediateAgg(GLOBAL_SQL_SUM, INTERMEDIATE_SQL_SUM);
+        addGlobalAgg(SQL_SUM, GLOBAL_SQL_SUM);
         addScalarAgg(SQL_SUM, SCALAR_SQL_SUM);
 
-        addSerialAgg(SQL_SUM, SERIAL_SQL_SUM);
-        addSerialAgg(LOCAL_SQL_SUM, SERIAL_LOCAL_SQL_SUM);
         addAgg(SERIAL_SQL_SUM);
         addAgg(SERIAL_LOCAL_SQL_SUM);
+        addAgg(SERIAL_GLOBAL_SQL_SUM);
+        addSerialAgg(SQL_SUM, SERIAL_SQL_SUM);
+        addSerialAgg(LOCAL_SQL_SUM, SERIAL_LOCAL_SQL_SUM);
+        addSerialAgg(GLOBAL_SQL_SUM, SERIAL_GLOBAL_SQL_SUM);
         addLocalAgg(SERIAL_SQL_SUM, SERIAL_LOCAL_SQL_SUM);
-        addIntermediateAgg(SERIAL_LOCAL_SQL_SUM, SERIAL_SQL_SUM);
         addIntermediateAgg(SERIAL_SQL_SUM, SERIAL_SQL_SUM);
-        addGlobalAgg(SERIAL_SQL_SUM, SERIAL_SQL_SUM);
-
-        // SQL SUM DISTINCT
+        addIntermediateAgg(SERIAL_LOCAL_SQL_SUM, SERIAL_INTERMEDIATE_SQL_SUM);
+        addIntermediateAgg(SERIAL_GLOBAL_SQL_SUM, SERIAL_INTERMEDIATE_SQL_SUM);
+        addGlobalAgg(SERIAL_SQL_SUM, SERIAL_GLOBAL_SQL_SUM);
 
+        // SQL SUM Distinct
         addDistinctAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM);
         addScalarAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM_DISTINCT);
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
deleted file mode 100644
index 32a6487..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.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.api.exceptions.SourceLocation;
-
-public class NumericAggTypeComputer extends AbstractResultTypeComputer {
-    public static final NumericAggTypeComputer INSTANCE = new NumericAggTypeComputer();
-
-    private NumericAggTypeComputer() {
-    }
-
-    @Override
-    protected void checkArgType(String funcName, 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, funcName, tag);
-        }
-    }
-
-    @Override
-    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        ATypeTag tag = strippedInputTypes[0].getTypeTag();
-        switch (tag) {
-            case DOUBLE:
-            case FLOAT:
-            case BIGINT:
-            case INTEGER:
-            case SMALLINT:
-            case TINYINT:
-            case ANY:
-                IAType type = strippedInputTypes[0];
-                return AUnionType.createNullableType(type, "AggResult");
-            default:
-                // All other possible cases.
-                return BuiltinType.ANULL;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7b0bca6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.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.api.exceptions.SourceLocation;
+
+public class NumericSumAggTypeComputer extends AbstractResultTypeComputer {
+    public static final NumericSumAggTypeComputer INSTANCE = new NumericSumAggTypeComputer();
+
+    private NumericSumAggTypeComputer() {
+    }
+
+    @Override
+    protected void checkArgType(String funcName, 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, funcName, tag);
+        }
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        ATypeTag tag = strippedInputTypes[0].getTypeTag();
+        switch (tag) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+                IAType int64Type = BuiltinType.AINT64;
+                return AUnionType.createNullableType(int64Type, "AggResult");
+            case FLOAT:
+            case DOUBLE:
+                IAType doubleType = BuiltinType.ADOUBLE;
+                return AUnionType.createNullableType(doubleType, "AggResult");
+            case ANY:
+                IAType anyType = strippedInputTypes[0];
+                return AUnionType.createNullableType(anyType, "AggResult");
+            default:
+                // All other possible cases.
+                return BuiltinType.ANULL;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
index 078ea57..e1fcd1a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
@@ -28,14 +28,8 @@ public class ScalarSqlSumAggregateDescriptor extends AbstractScalarAggregateDesc
 
     private static final long serialVersionUID = 1L;
 
-    public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_SUM;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlSumAggregateDescriptor(SqlSumAggregateDescriptor.FACTORY.createFunctionDescriptor());
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY =
+            () -> new ScalarSqlSumAggregateDescriptor(SqlSumAggregateDescriptor.FACTORY.createFunctionDescriptor());
 
     private ScalarSqlSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
         super(aggFuncDesc);
@@ -43,6 +37,6 @@ public class ScalarSqlSumAggregateDescriptor extends AbstractScalarAggregateDesc
 
     @Override
     public FunctionIdentifier getIdentifier() {
-        return FID;
+        return BuiltinFunctions.SCALAR_SQL_SUM;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
index 4d547e1..eb1d38b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
@@ -28,14 +28,8 @@ public class ScalarSumAggregateDescriptor extends AbstractScalarAggregateDescrip
 
     private static final long serialVersionUID = 1L;
 
-    public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SUM;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSumAggregateDescriptor(SumAggregateDescriptor.FACTORY.createFunctionDescriptor());
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY =
+            () -> new ScalarSumAggregateDescriptor(SumAggregateDescriptor.FACTORY.createFunctionDescriptor());
 
     private ScalarSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
         super(aggFuncDesc);
@@ -43,6 +37,6 @@ public class ScalarSumAggregateDescriptor extends AbstractScalarAggregateDescrip
 
     @Override
     public FunctionIdentifier getIdentifier() {
-        return FID;
+        return BuiltinFunctions.SCALAR_SUM;
     }
 }


[2/5] asterixdb git commit: [ASTERIXDB-2460][FUN] Fix sum() overflow bug

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
index 132fd54..3c07b47 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
@@ -18,30 +18,18 @@
  */
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.dataflow.data.nontagged.serde.*;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableDouble;
-import org.apache.asterix.om.base.AMutableFloat;
-import org.apache.asterix.om.base.AMutableInt16;
-import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.AMutableInt8;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -52,187 +40,229 @@ import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 public abstract class AbstractSerializableSumAggregateFunction extends AbstractSerializableAggregateFunction {
+
+    // Handles evaluating and storing/passing serialized data
     protected static final int AGG_TYPE_OFFSET = 0;
     private static final int SUM_OFFSET = 1;
-
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
-    private AMutableDouble aDouble = new AMutableDouble(0);
-    private AMutableFloat aFloat = new AMutableFloat(0);
+
+    // Aggregate type
+    protected ATypeTag aggType;
+
+    // Result holders
     private AMutableInt64 aInt64 = new AMutableInt64(0);
-    private AMutableInt32 aInt32 = new AMutableInt32(0);
-    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+    private AMutableDouble aDouble = new AMutableDouble(0);
+
+    // Flags for output type (If all output flags are false, double output is used)
+    private boolean isUseInt64ForResult = true;
+
+    // Serializer/Deserializer
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer aInt64Serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
     @SuppressWarnings("rawtypes")
-    public ISerializerDeserializer serde;
+    private ISerializerDeserializer aDoubleSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
 
+    // Constructor
     public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
             SourceLocation sourceLoc) throws HyracksDataException {
         super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
+    // Abstract methods
+    protected abstract boolean skipStep(byte[] state, int start); // Skip step
+
+    protected abstract void processNull(byte[] state, int start); // Handle NULL step
+
+    protected abstract void processSystemNull() throws HyracksDataException; // Handle SYSTEM_NULL step
+
+    protected abstract void finishNull(DataOutput out) throws IOException; // Handle NULL finish
+
+    protected abstract void finishSystemNull(DataOutput out) throws IOException; // Handle SYSTEM_NULL finish
+
+    // Init the values
     @Override
     public void init(DataOutput state) throws HyracksDataException {
         try {
             state.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
-            state.writeDouble(0.0);
+            state.writeLong(0);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
     }
 
+    // Called for each incoming tuple
     @Override
     public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        // Skip current step
         if (skipStep(state, start)) {
             return;
         }
-        ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
-        double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
+
+        // Evaluate/Get the data from the tuple
         eval.evaluate(tuple, inputVal);
         byte[] bytes = inputVal.getByteArray();
         int offset = inputVal.getStartOffset();
 
+        // Get the data type tag
         ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+
+        // Handle MISSING and NULL values
         if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
             processNull(state, start);
             return;
-        } else if (aggType == ATypeTag.SYSTEM_NULL) {
-            aggType = typeTag;
-        } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            if (typeTag.ordinal() > aggType.ordinal()) {
-                throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset],
-                        aggType.serialize());
-            } else {
-                throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize(),
-                        bytes[offset]);
-            }
-        }
-
-        if (ATypeHierarchy.canPromote(aggType, typeTag)) {
-            aggType = typeTag;
         }
 
+        // Calculate based on the incoming data type + handles invalid data type
         switch (typeTag) {
             case TINYINT: {
                 byte val = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
-                sum += val;
+                processInt64Value(state, start, val);
                 break;
             }
             case SMALLINT: {
                 short val = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
-                sum += val;
+                processInt64Value(state, start, val);
                 break;
             }
             case INTEGER: {
                 int val = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
-                sum += val;
+                processInt64Value(state, start, val);
                 break;
             }
             case BIGINT: {
                 long val = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
-                sum += val;
+                processInt64Value(state, start, val);
                 break;
             }
             case FLOAT: {
+                upgradeOutputType();
                 float val = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
-                sum += val;
+                processFloatValue(state, start, val);
                 break;
             }
             case DOUBLE: {
+                upgradeOutputType();
                 double val = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
-                sum += val;
+                processFloatValue(state, start, val);
                 break;
             }
-            case NULL: {
-                aggType = typeTag;
-                break;
-            }
-            case SYSTEM_NULL: {
+            case SYSTEM_NULL:
                 processSystemNull();
                 break;
-            }
             default:
                 throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset]);
         }
-        state[start + AGG_TYPE_OFFSET] = aggType.serialize();
+    }
+
+    // Upgrade the output type
+    private void upgradeOutputType() {
+        isUseInt64ForResult = false;
+    }
+
+    // Process int64 value
+    private void processInt64Value(byte[] state, int start, long value) throws HyracksDataException {
+        // Check the output flag first
+        if (!isUseInt64ForResult) {
+            processFloatValue(state, start, value);
+        }
+
+        // Int64 output, watch out for overflow exception
+        else {
+            try {
+                // Current total
+                long sum = BufferSerDeUtil.getLong(state, start + SUM_OFFSET);
+                sum = Math.addExact(sum, value);
+
+                // Write the output
+                state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_INT64_TYPE_TAG;
+                BufferSerDeUtil.writeLong(sum, state, start + SUM_OFFSET);
+            } catch (ArithmeticException ignored) {
+                throw new OverflowException(sourceLoc, getIdentifier());
+            }
+        }
+    }
+
+    // Process float value
+    private void processFloatValue(byte[] state, int start, double value) {
+        double sum;
+        aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
+
+        // This checks if the previous written value is bigint, SYSTEM_NULL or double and reads it accordingly
+        // Important: SYSTEM_NULL reads int64 because the written value is in64 as well, check the init() method
+        if (aggType == ATypeTag.BIGINT || aggType == ATypeTag.SYSTEM_NULL) {
+            // Last write was a bigint or SYSTEM_NULL
+            sum = BufferSerDeUtil.getLong(state, start + SUM_OFFSET);
+        } else {
+            // Last write was a double
+            sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
+        }
+
+        // Add the value
+        sum += value;
+
+        // Write the output
+        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG;
         BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
     }
 
     @SuppressWarnings("unchecked")
     @Override
+    public void finishPartial(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        // finishPartial() has identical behavior to finish()
+        finishFinal(state, start, len, out);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
     public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
-        ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
-        double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
+        // Finish
+        finishFinal(state, start, len, out);
+    }
+
+    @SuppressWarnings({ "unchecked", "unused" })
+    private void finishFinal(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
+
         try {
-            switch (aggType) {
-                case TINYINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
-                    aInt8.setValue((byte) sum);
-                    serde.serialize(aInt8, out);
-                    break;
+            // aggType is SYSTEM_NULL (ran over zero values)
+            if (aggType == ATypeTag.SYSTEM_NULL) {
+                if (GlobalConfig.DEBUG) {
+                    GlobalConfig.ASTERIX_LOGGER.trace("SUM aggregate ran over zero values.");
                 }
-                case SMALLINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
-                    aInt16.setValue((short) sum);
-                    serde.serialize(aInt16, out);
-                    break;
-                }
-                case INTEGER: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-                    aInt32.setValue((int) sum);
-                    serde.serialize(aInt32, out);
-                    break;
-                }
-                case BIGINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-                    aInt64.setValue((long) sum);
-                    serde.serialize(aInt64, out);
-                    break;
-                }
-                case FLOAT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
-                    aFloat.setValue((float) sum);
-                    serde.serialize(aFloat, out);
-                    break;
-                }
-                case DOUBLE: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                finishSystemNull(out);
+            }
+            // aggType is NULL
+            else if (aggType == ATypeTag.NULL) {
+                finishNull(out);
+            }
+            // Pass the result
+            else {
+                if (isUseInt64ForResult) {
+                    long sum = BufferSerDeUtil.getLong(state, start + SUM_OFFSET);
+                    aInt64.setValue(sum);
+                    aInt64Serde.serialize(aInt64, out);
+                } else {
+                    double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
                     aDouble.setValue(sum);
-                    serde.serialize(aDouble, out);
-                    break;
-                }
-                case NULL: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-                    serde.serialize(ANull.NULL, out);
-                    break;
-                }
-                case SYSTEM_NULL: {
-                    finishSystemNull(out);
-                    break;
+                    aDoubleSerde.serialize(aDouble, out);
                 }
-                default:
-                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
     }
 
-    @Override
-    public void finishPartial(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
-        finish(state, start, len, out);
-    }
-
-    protected boolean skipStep(byte[] state, int start) {
-        return false;
+    // Function identifier
+    private FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SUM;
     }
-
-    protected abstract void processNull(byte[] state, int start);
-
-    protected abstract void processSystemNull() throws HyracksDataException;
-
-    protected abstract void finishSystemNull(DataOutput out) throws IOException;
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateDescriptor.java
new file mode 100644
index 0000000..3e74308
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SerializableGlobalSqlSumAggregateDescriptor
+        extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_GLOBAL_SQL_SUM;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new SerializableGlobalSqlSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateFunction.java
new file mode 100644
index 0000000..2d8fa61
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSumAggregateFunction.java
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableGlobalSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableGlobalSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finish(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateDescriptor.java
new file mode 100644
index 0000000..24bfcc9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SerializableGlobalSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_GLOBAL_SUM;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new SerializableGlobalSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateFunction.java
new file mode 100644
index 0000000..35e2f89
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableGlobalSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableGlobalSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finish(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_NULL_TYPE_TAG;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateDescriptor.java
new file mode 100644
index 0000000..8691756
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SerializableIntermediateSqlSumAggregateDescriptor
+        extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSqlSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_INTERMEDIATE_SQL_SUM;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new SerializableIntermediateSqlSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateFunction.java
new file mode 100644
index 0000000..04eeed3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSumAggregateFunction.java
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableIntermediateSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableIntermediateSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finishPartial(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateDescriptor.java
new file mode 100644
index 0000000..d1d2c0f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SerializableIntermediateSumAggregateDescriptor
+        extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSumAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_INTERMEDIATE_SUM;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new SerializableIntermediateSumAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateFunction.java
new file mode 100644
index 0000000..4adc845
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSumAggregateFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableIntermediateSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableIntermediateSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finishPartial(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_NULL_TYPE_TAG;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
index 3798b49..79b31c4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class SerializableLocalSqlSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SerializableLocalSqlSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -53,7 +47,7 @@ public class SerializableLocalSqlSumAggregateDescriptor extends AbstractSerializ
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, true, ctx, sourceLoc);
+                return new SerializableLocalSqlSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateFunction.java
new file mode 100644
index 0000000..9aea883
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateFunction.java
@@ -0,0 +1,82 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableLocalSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableLocalSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finishPartial(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        return false;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        // Do nothing
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() throws HyracksDataException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
index 54d24f7..512d211 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
@@ -19,25 +19,19 @@
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class SerializableLocalSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SerializableLocalSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -53,7 +47,7 @@ public class SerializableLocalSumAggregateDescriptor extends AbstractSerializabl
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, true, ctx, sourceLoc);
+                return new SerializableLocalSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateFunction.java
new file mode 100644
index 0000000..8a9e0f9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateFunction.java
@@ -0,0 +1,84 @@
+/*
+ * 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.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SerializableLocalSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
+
+    public SerializableLocalSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finishPartial(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
+        return aggType == ATypeTag.NULL;
+    }
+
+    // Handle NULL step
+    @Override
+    protected void processNull(byte[] state, int start) {
+        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_NULL_TYPE_TAG;
+    }
+
+    // Handle SYSTEM_NULL step
+    @Override
+    protected void processSystemNull() throws HyracksDataException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
+
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
index d51a6fc..0dc5e17 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class SerializableSqlSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SerializableSqlSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SerializableSqlSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -53,7 +47,7 @@ public class SerializableSqlSumAggregateDescriptor extends AbstractSerializableA
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, false, ctx, sourceLoc);
+                return new SerializableSqlSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
index 38033f0..3062a37 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
@@ -18,54 +18,62 @@
  */
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
 
 public class SerializableSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
-    private final boolean isLocalAgg;
 
-    public SerializableSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+    public SerializableSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
         super(args, context, sourceLoc);
-        this.isLocalAgg = isLocalAgg;
     }
 
+    // Called for each incoming tuple
+    @Override
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finish(state, start, len, out);
+    }
+
+    // Is skip
+    @Override
+    protected boolean skipStep(byte[] state, int start) {
+        return false;
+    }
+
+    // Handle NULL step
     @Override
     protected void processNull(byte[] state, int start) {
+        // Do nothing
     }
 
+    // Handle SYSTEM_NULL step
     @Override
-    protected void processSystemNull() throws HyracksDataException {
-        // For global aggregates simply ignore system null here,
-        // but if all input value are system null, then we should return
-        // null in finish().
-        if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
-                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
-        }
+    protected void processSystemNull() {
+        // Do nothing
     }
 
-    @SuppressWarnings("unchecked")
+    // Handle NULL finish
     @Override
-    protected void finishSystemNull(DataOutput out) throws IOException {
-        // Empty stream. For local agg return system null. For global agg return null.
-        if (isLocalAgg) {
-            out.writeByte(ATypeTag.SYSTEM_NULL.serialize());
-        } else {
-            serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-            serde.serialize(ANull.NULL, out);
-        }
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
     }
 
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
index 43eea5b..7619ca8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class SerializableSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new SerializableSumAggregateDescriptor();
-        }
-    };
+    public static final IFunctionDescriptorFactory FACTORY = SerializableSumAggregateDescriptor::new;
 
     @Override
     public FunctionIdentifier getIdentifier() {
@@ -53,7 +47,7 @@ public class SerializableSumAggregateDescriptor extends AbstractSerializableAggr
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, false, ctx, sourceLoc);
+                return new SerializableSumAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
index 278914f..0be49e7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
@@ -18,62 +18,64 @@
  */
 package org.apache.asterix.runtime.aggregates.serializable.std;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
 
 public class SerializableSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
-    private final boolean isLocalAgg;
 
-    public SerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+    public SerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
         super(args, context, sourceLoc);
-        this.isLocalAgg = isLocalAgg;
     }
 
+    // Called for each incoming tuple
     @Override
-    protected void processNull(byte[] state, int start) {
-        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_NULL_TYPE_TAG;
+    public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
+        super.step(tuple, state, start, len);
+    }
+
+    // Finish calculation
+    @Override
+    public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
+        super.finish(state, start, len, out);
     }
 
+    // Is skip
     @Override
     protected boolean skipStep(byte[] state, int start) {
         ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
         return aggType == ATypeTag.NULL;
     }
 
+    // Handle NULL step
     @Override
-    protected void processSystemNull() throws HyracksDataException {
-        // For global aggregates simply ignore system null here,
-        // but if all input value are system null, then we should return
-        // null in finish().
-        if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
-                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
-        }
+    protected void processNull(byte[] state, int start) {
+        state[start + AGG_TYPE_OFFSET] = ATypeTag.SERIALIZED_NULL_TYPE_TAG;
     }
 
-    @SuppressWarnings("unchecked")
+    // Handle SYSTEM_NULL step
     @Override
-    protected void finishSystemNull(DataOutput out) throws IOException {
-        // Empty stream. For local agg return system null. For global agg return null.
-        if (isLocalAgg) {
-            out.writeByte(ATypeTag.SYSTEM_NULL.serialize());
-        } else {
-            serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-            serde.serialize(ANull.NULL, out);
-        }
+    protected void processSystemNull() {
+        // Do nothing
+    }
+
+    // Handle NULL finish
+    @Override
+    protected void finishNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
     }
 
+    // Handle SYSTEM_NULL finish
+    @Override
+    protected void finishSystemNull(DataOutput out) throws IOException {
+        out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
index 037e307..836c24e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
@@ -18,29 +18,18 @@
  */
 package org.apache.asterix.runtime.aggregates.std;
 
-import java.io.IOException;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.dataflow.data.nontagged.serde.*;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableDouble;
-import org.apache.asterix.om.base.AMutableFloat;
-import org.apache.asterix.om.base.AMutableInt16;
-import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.AMutableInt8;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -52,91 +41,119 @@ import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
+import java.io.IOException;
+
 public abstract class AbstractSumAggregateFunction extends AbstractAggregateFunction {
+
+    // Handles evaluating and storing/passing serialized data
     protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
-    private double sum;
+
+    // Aggregate type
     protected ATypeTag aggType;
-    private AMutableDouble aDouble = new AMutableDouble(0);
-    private AMutableFloat aFloat = new AMutableFloat(0);
+
+    // Result holders
+    private long sumInt64;
+    private double sumDouble;
     private AMutableInt64 aInt64 = new AMutableInt64(0);
-    private AMutableInt32 aInt32 = new AMutableInt32(0);
-    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+    private AMutableDouble aDouble = new AMutableDouble(0);
+
+    // Flags for output type (If all output flags are false, double output is used)
+    private boolean isUseInt64ForResult = true;
+
+    // Serializer/Deserializer
     @SuppressWarnings("rawtypes")
-    protected ISerializerDeserializer serde;
+    private ISerializerDeserializer aInt64Serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer aDoubleSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
 
+    // Constructor
     public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
             SourceLocation sourceLoc) throws HyracksDataException {
         super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
+    // Abstract methods
+    protected abstract boolean skipStep(); // Skip step
+
+    protected abstract void processNull(); // Handle NULL step
+
+    protected abstract void processSystemNull() throws HyracksDataException; // Handle SYSTEM_NULL step
+
+    protected abstract void finishNull(IPointable result) throws IOException; // Handle NULL finish
+
+    protected abstract void finishSystemNull(IPointable result) throws IOException; // Handle SYSTEM_NULL finish
+
+    // Init the values
     @Override
     public void init() throws HyracksDataException {
         aggType = ATypeTag.SYSTEM_NULL;
-        sum = 0.0;
+        sumInt64 = 0;
+        sumDouble = 0.0;
     }
 
+    // Called for each incoming tuple
     @Override
     public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        // Skip current step
         if (skipStep()) {
             return;
         }
+
+        // Evaluate/Get the data from the tuple
         eval.evaluate(tuple, inputVal);
         byte[] data = inputVal.getByteArray();
         int offset = inputVal.getStartOffset();
 
+        // Get the data type tag
         ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]);
+
+        // Handle MISSING and NULL values
         if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
             processNull();
             return;
-        } else if (aggType == ATypeTag.SYSTEM_NULL) {
-            aggType = typeTag;
-        } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            if (typeTag.ordinal() > aggType.ordinal()) {
-                throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, typeTag.serialize(),
-                        aggType.serialize());
-            } else {
-                throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize(),
-                        typeTag.serialize());
-            }
         }
-
-        if (ATypeHierarchy.canPromote(aggType, typeTag)) {
+        // Non-missing and Non-null
+        else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         }
 
+        // Calculate based on the incoming data type + handles invalid data type
         switch (typeTag) {
             case TINYINT: {
                 byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
-                sum += val;
+                processInt64Value(val);
                 break;
             }
             case SMALLINT: {
                 short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
-                sum += val;
+                processInt64Value(val);
                 break;
             }
             case INTEGER: {
                 int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
-                sum += val;
+                processInt64Value(val);
                 break;
             }
             case BIGINT: {
                 long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
-                sum += val;
+                processInt64Value(val);
                 break;
             }
             case FLOAT: {
+                upgradeOutputType();
                 float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
-                sum += val;
+                processFloatValue(val);
                 break;
             }
             case DOUBLE: {
+                upgradeOutputType();
                 double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
-                sum += val;
+                processFloatValue(val);
                 break;
             }
             case SYSTEM_NULL: {
@@ -144,83 +161,95 @@ public abstract class AbstractSumAggregateFunction extends AbstractAggregateFunc
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, typeTag.serialize());
             }
         }
     }
 
-    @SuppressWarnings("unchecked")
-    @Override
-    public void finish(IPointable result) throws HyracksDataException {
-        resultStorage.reset();
-        try {
-            switch (aggType) {
-                case TINYINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
-                    aInt8.setValue((byte) sum);
-                    serde.serialize(aInt8, resultStorage.getDataOutput());
-                    break;
-                }
-                case SMALLINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
-                    aInt16.setValue((short) sum);
-                    serde.serialize(aInt16, resultStorage.getDataOutput());
-                    break;
-                }
-                case INTEGER: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-                    aInt32.setValue((int) sum);
-                    serde.serialize(aInt32, resultStorage.getDataOutput());
-                    break;
-                }
-                case BIGINT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-                    aInt64.setValue((long) sum);
-                    serde.serialize(aInt64, resultStorage.getDataOutput());
-                    break;
-                }
-                case FLOAT: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
-                    aFloat.setValue((float) sum);
-                    serde.serialize(aFloat, resultStorage.getDataOutput());
-                    break;
-                }
-                case DOUBLE: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
-                    aDouble.setValue(sum);
-                    serde.serialize(aDouble, resultStorage.getDataOutput());
-                    break;
-                }
-                case NULL: {
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-                    serde.serialize(ANull.NULL, resultStorage.getDataOutput());
-                    break;
-                }
-                case SYSTEM_NULL: {
-                    finishSystemNull();
-                    break;
-                }
-                default:
-                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
+    // Upgrade the output type
+    private void upgradeOutputType() {
+        isUseInt64ForResult = false;
+    }
+
+    // Process int64 value
+    private void processInt64Value(long value) throws HyracksDataException {
+        // Check the output flag first
+        if (!isUseInt64ForResult) {
+            processFloatValue(value);
+        }
+        // Int64 output, watch out for overflow exception
+        else {
+            try {
+                sumInt64 = Math.addExact(sumInt64, value);
+                sumDouble = sumInt64; // Keep the sumDouble variable up-to-date as well
+            } catch (ArithmeticException ignored) {
+                throw new OverflowException(sourceLoc, getIdentifier());
             }
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
         }
-        result.set(resultStorage);
     }
 
+    // Process float value
+    private void processFloatValue(double value) {
+        // If this method is called, it means the output is going to be a double, no need to check the output type flag
+        // and the sumInt64 can be ignored
+        sumDouble += value;
+    }
+
+    // Called for partial calculations
+    @SuppressWarnings("unchecked")
     @Override
     public void finishPartial(IPointable result) throws HyracksDataException {
-        finish(result);
+        // finishPartial() has identical behavior to finish()
+        finishFinal(result);
     }
 
-    protected boolean skipStep() {
-        return false;
+    // Called for final calculations
+    @SuppressWarnings("unchecked")
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        // Finish
+        finishFinal(result);
     }
 
-    protected abstract void processNull();
+    // Called for final calculations
+    @SuppressWarnings("unchecked")
+    private void finishFinal(IPointable result) throws HyracksDataException {
+        // Reset the result storage
+        resultStorage.reset();
+
+        try {
+            // aggType is SYSTEM_NULL
+            if (aggType == ATypeTag.SYSTEM_NULL) {
+                if (GlobalConfig.DEBUG) {
+                    GlobalConfig.ASTERIX_LOGGER.trace("SUM aggregate ran over zero values.");
+                }
 
-    protected abstract void processSystemNull() throws HyracksDataException;
+                finishSystemNull(result);
+            }
+            // aggType is NULL
+            else if (aggType == ATypeTag.NULL) {
+                finishNull(result);
+            }
+            // Pass the result
+            else {
+                // Output type based on the flag
+                if (isUseInt64ForResult) {
+                    aInt64.setValue(sumInt64);
+                    aInt64Serde.serialize(aInt64, resultStorage.getDataOutput());
+                    result.set(resultStorage);
+                } else {
+                    aDouble.setValue(sumDouble);
+                    aDoubleSerde.serialize(aDouble, resultStorage.getDataOutput());
+                    result.set(resultStorage);
+                }
+            }
+        } catch (IOException ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
 
-    protected abstract void finishSystemNull() throws IOException;
+    // Function identifier
+    private FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SUM;
+    }
 }


[4/5] asterixdb git commit: [ASTERIXDB-2460][FUN] Fix sum() overflow bug

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
new file mode 100644
index 0000000..8c0b731
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':float('2.0')};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':'hello world'};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double('3.0')};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
new file mode 100644
index 0000000..36d1701
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_sum_mixed/serial_sum_mixed.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+ /*
+ * Description  : Run serial-sum over an ordered list with mixed types
+ * Expected Res : Failure
+ * Date         : March 5th 2018
+ */
+
+use test;
+
+select gid, array_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid 
+group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
new file mode 100644
index 0000000..21479a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.1.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
new file mode 100644
index 0000000..2f0f56b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int64_overflow/sum_int64_overflow.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+select element test.array_sum((
+    select element x
+    from  [test.bigint('1'),test.bigint('2'),test.bigint('3'),test.bigint('9223372036854775807')] as x
+));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int8/sum_int8.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int8/sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int8/sum_int8.3.query.sqlpp
index f52e663..64c43ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int8/sum_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_int8/sum_int8.3.query.sqlpp
@@ -22,5 +22,5 @@ use test;
 
 select element test.array_sum((
     select element x
-    from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
+    from  [test.tinyint('100'),test.tinyint('100'),test.tinyint('100')] as x
 ));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double/serial_sum_double.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_double_null/serial_sum_double_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.2.update.sqlpp
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_empty/serial_sum_empty.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : float
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.2.update.sqlpp
new file mode 100644
index 0000000..c83ba1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('473847')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('38473827484738239')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('678900')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float/serial_sum_float.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : float
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
new file mode 100644
index 0000000..c83ba1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('473847')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('38473827484738239')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('678900')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_float_null/serial_sum_float_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : smallint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16/serial_sum_int16.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : smallint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int16_null/serial_sum_int16_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : integer
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32/serial_sum_int32.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : integer
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int32_null/serial_sum_int32_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64/serial_sum_int64.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_null/serial_sum_int64_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : bigint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
new file mode 100644
index 0000000..5efbe51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.2.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
+insert into Test
+select element {'id':5,'gid':1,'val':bigint('9223372036854775807'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int64_overflow/serial_sum_int64_overflow.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : tinyint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.3.query.sqlpp
new file mode 100644
index 0000000..f027a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8/serial_sum_int8.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.val from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : bigint,
+  gid : bigint,
+  val : tinyint
+};
+
+create  dataset Test(TestType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5af85d9e/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
new file mode 100644
index 0000000..df2f905
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_sum_int8_null/serial_sum_int8_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_sum((select value g.valplus from g)) as sum
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);