You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/05/24 01:31:35 UTC

[17/22] incubator-asterixdb git commit: ASTERIXDB-1228: Add MISSING into the data model.

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ARecordPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ARecordPrinter.java
index e6eb12c..5388837 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ARecordPrinter.java
@@ -45,8 +45,8 @@ class ARecordPrinter {
 
     }
 
-    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor) throws IOException,
-            AsterixException {
+    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor)
+            throws IOException, AsterixException {
         List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
         List<IVisitablePointable> fieldTags = recordAccessor.getFieldTypeTags();
         List<IVisitablePointable> fieldValues = recordAccessor.getFieldValues();
@@ -71,8 +71,8 @@ class ARecordPrinter {
             List<IVisitablePointable> fieldTags, List<IVisitablePointable> fieldValues, int i) throws AsterixException {
         IVisitablePointable itemTypeTag = fieldTags.get(i);
         IVisitablePointable item = fieldValues.get(i);
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag.getByteArray()[itemTypeTag
-                .getStartOffset()]);
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
         itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
 
         // print field value

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/AListPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/AListPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/AListPrinter.java
index 892a8b6..826c465 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/AListPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/AListPrinter.java
@@ -44,8 +44,8 @@ class AListPrinter {
     public AListPrinter(boolean ordered) {
     }
 
-    public void printList(AListVisitablePointable listAccessor, PrintStream ps, APrintVisitor visitor) throws IOException,
-            AsterixException {
+    public void printList(AListVisitablePointable listAccessor, PrintStream ps, APrintVisitor visitor)
+            throws IOException, AsterixException {
         List<IVisitablePointable> itemTags = listAccessor.getItemTags();
         List<IVisitablePointable> items = listAccessor.getItems();
         itemVisitorArg.first = ps;
@@ -73,8 +73,8 @@ class AListPrinter {
             int i) throws AsterixException {
         IVisitablePointable itemTypeTag = itemTags.get(i);
         IVisitablePointable item = items.get(i);
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag.getByteArray()[itemTypeTag
-                .getStartOffset()]);
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
         itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
         item.accept(visitor, itemVisitorArg);
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/APrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/APrintVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/APrintVisitor.java
index ac0b73a..81677f1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/APrintVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/APrintVisitor.java
@@ -19,6 +19,11 @@
 
 package org.apache.asterix.om.pointables.printer.json.clean;
 
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.printers.adm.ShortWithoutTypeInfoPrinter;
 import org.apache.asterix.dataflow.data.nontagged.printers.json.clean.ABinaryHexPrinter;
@@ -54,11 +59,6 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * This class is a IVisitablePointableVisitor implementation which recursively
  * visit a given record, list or flat value of a given type, and print it to a
@@ -124,6 +124,7 @@ public class APrintVisitor implements IVisitablePointableVisitor<Void, Pair<Prin
                     AInt64Printer.INSTANCE.print(b, s, l, ps);
                     break;
                 }
+                case MISSING:
                 case NULL: {
                     ANullPrinter.INSTANCE.print(b, s, l, ps);
                     break;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/ARecordPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/ARecordPrinter.java
index 24815b0..6644f3a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/clean/ARecordPrinter.java
@@ -47,8 +47,8 @@ class ARecordPrinter {
 
     }
 
-    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor) throws IOException,
-            AsterixException {
+    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor)
+            throws IOException, AsterixException {
         List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
         List<IVisitablePointable> fieldTags = recordAccessor.getFieldTypeTags();
         List<IVisitablePointable> fieldValues = recordAccessor.getFieldValues();
@@ -79,8 +79,8 @@ class ARecordPrinter {
             List<IVisitablePointable> fieldTags, List<IVisitablePointable> fieldValues, int i) throws AsterixException {
         IVisitablePointable itemTypeTag = fieldTags.get(i);
         IVisitablePointable item = fieldValues.get(i);
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag.getByteArray()[itemTypeTag
-                .getStartOffset()]);
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
         itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
 
         // print field name

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/AListPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/AListPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/AListPrinter.java
index 435a674..3b6310a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/AListPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/AListPrinter.java
@@ -49,8 +49,8 @@ class AListPrinter {
         }
     }
 
-    public void printList(AListVisitablePointable listAccessor, PrintStream ps, APrintVisitor visitor) throws IOException,
-            AsterixException {
+    public void printList(AListVisitablePointable listAccessor, PrintStream ps, APrintVisitor visitor)
+            throws IOException, AsterixException {
         List<IVisitablePointable> itemTags = listAccessor.getItemTags();
         List<IVisitablePointable> items = listAccessor.getItems();
         itemVisitorArg.first = ps;
@@ -78,8 +78,8 @@ class AListPrinter {
             int i) throws AsterixException {
         IVisitablePointable itemTypeTag = itemTags.get(i);
         IVisitablePointable item = items.get(i);
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag.getByteArray()[itemTypeTag
-                .getStartOffset()]);
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
         itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
         item.accept(visitor, itemVisitorArg);
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/APrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/APrintVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/APrintVisitor.java
index 6a96fcd..c0f3d59 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/APrintVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/APrintVisitor.java
@@ -19,6 +19,11 @@
 
 package org.apache.asterix.om.pointables.printer.json.lossless;
 
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.printers.adm.ShortWithoutTypeInfoPrinter;
 import org.apache.asterix.dataflow.data.nontagged.printers.json.lossless.ABinaryHexPrinter;
@@ -54,11 +59,6 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * This class is a IVisitablePointableVisitor implementation which recursively
  * visit a given record, list or flat value of a given type, and print it to a
@@ -124,6 +124,7 @@ public class APrintVisitor implements IVisitablePointableVisitor<Void, Pair<Prin
                     AInt64Printer.INSTANCE.print(b, s, l, ps);
                     break;
                 }
+                case MISSING:
                 case NULL: {
                     ANullPrinter.INSTANCE.print(b, s, l, ps);
                     break;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/ARecordPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/ARecordPrinter.java
index e111f76..fb7d6e1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/lossless/ARecordPrinter.java
@@ -47,8 +47,8 @@ class ARecordPrinter {
 
     }
 
-    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor) throws IOException,
-            AsterixException {
+    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, APrintVisitor visitor)
+            throws IOException, AsterixException {
         List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
         List<IVisitablePointable> fieldTags = recordAccessor.getFieldTypeTags();
         List<IVisitablePointable> fieldValues = recordAccessor.getFieldValues();
@@ -79,8 +79,8 @@ class ARecordPrinter {
             List<IVisitablePointable> fieldTags, List<IVisitablePointable> fieldValues, int i) throws AsterixException {
         IVisitablePointable itemTypeTag = fieldTags.get(i);
         IVisitablePointable item = fieldValues.get(i);
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag.getByteArray()[itemTypeTag
-                .getStartOffset()]);
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
         itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
 
         // print field name

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
new file mode 100644
index 0000000..f3e0472
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.base;
+
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * This abstract class takes care of the handling of optional types.
+ * If a subclass follows the MISSING-in-MISSING-out and NULL-in-NULL-out semantics,
+ * then it only needs to think of non-optional types and this abstract class
+ * will strip the input types and wrap the output type.
+ */
+public abstract class AbstractResultTypeComputer implements IResultTypeComputer {
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        return TypeComputeUtils.resolveResultType(expression, env, (index, type) -> checkArgType(index, type),
+                types -> getResultType(types), true);
+    }
+
+    /**
+     * Checks whether an input type violates the requirement.
+     *
+     * @param argIndex,
+     *            the index of the argument to consider.
+     * @param type,
+     *            the type of the input argument.
+     * @throws AlgebricksException
+     */
+    protected void checkArgType(int argIndex, IAType type) throws AlgebricksException {
+
+    }
+
+    /**
+     * Returns the result type without considering optional types.
+     *
+     * @param strippedInputTypes,
+     *            the stripped input types.
+     * @return the result type without considering optional types.
+     */
+    protected abstract IAType getResultType(IAType... strippedInputTypes);
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
new file mode 100644
index 0000000..04883e4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
@@ -0,0 +1,57 @@
+/*
+ * 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.base;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+public class TypeCastUtils {
+
+    private TypeCastUtils() {
+    }
+
+    public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredRecordType,
+            IAType inputRecordType) {
+        boolean changed = false;
+        Object[] opaqueParameters = expr.getOpaqueParameters();
+        if (opaqueParameters == null) {
+            opaqueParameters = new Object[2];
+            opaqueParameters[0] = requiredRecordType;
+            opaqueParameters[1] = inputRecordType;
+            expr.setOpaqueParameters(opaqueParameters);
+            changed = true;
+        }
+        return changed;
+    }
+
+    public static void resetRequiredAndInputTypes(AbstractFunctionCallExpression expr) {
+        expr.setOpaqueParameters(null);
+    }
+
+    public static IAType getRequiredType(AbstractFunctionCallExpression expr) {
+        Object[] type = expr.getOpaqueParameters();
+        if (type != null) {
+            IAType returnType = (IAType) type[0];
+            return returnType;
+        } else {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeComputerUtilities.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeComputerUtilities.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeComputerUtilities.java
deleted file mode 100644
index a3d8003..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeComputerUtilities.java
+++ /dev/null
@@ -1,124 +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.base;
-
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-
-public class TypeComputerUtilities {
-
-    public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredRecordType,
-            IAType inputRecordType) {
-        boolean changed = false;
-        Object[] opaqueParameters = expr.getOpaqueParameters();
-        if (opaqueParameters == null) {
-            opaqueParameters = new Object[2];
-            opaqueParameters[0] = requiredRecordType;
-            opaqueParameters[1] = inputRecordType;
-            expr.setOpaqueParameters(opaqueParameters);
-            changed = true;
-        }
-        return changed;
-    }
-
-    public static void resetRequiredAndInputTypes(AbstractFunctionCallExpression expr) {
-        expr.setOpaqueParameters(null);
-    }
-
-    public static IAType getRequiredType(AbstractFunctionCallExpression expr) {
-        Object[] type = expr.getOpaqueParameters();
-        if (type != null) {
-            IAType returnType = (IAType) type[0];
-            return returnType;
-        } else
-            return null;
-    }
-
-    public static IAType getInputType(AbstractFunctionCallExpression expr) {
-        Object[] type = expr.getOpaqueParameters();
-        if (type != null) {
-            IAType returnType = (IAType) type[1];
-            return returnType;
-        } else
-            return null;
-    }
-
-    public static boolean inputInferednullableType(ILogicalExpression expression, IVariableTypeEnvironment env)
-            throws AlgebricksException {
-        AbstractFunctionCallExpression func = (AbstractFunctionCallExpression) expression;
-        if (!(func instanceof ScalarFunctionCallExpression)) {
-            return true;
-        }
-        List<Mutable<ILogicalExpression>> args = func.getArguments();
-        for (Mutable<ILogicalExpression> arg : args) {
-            IAType type = (IAType) env.getType(arg.getValue());
-            if (type.getTypeTag() == ATypeTag.UNION || type.getTypeTag() == ATypeTag.NULL
-                    || type.getTypeTag() == ATypeTag.ANY) {
-                return true;
-            }
-            if (type.getTypeTag() == ATypeTag.RECORD || type.getTypeTag() == ATypeTag.UNORDEREDLIST
-                    || type.getTypeTag() == ATypeTag.ORDEREDLIST) {
-                if (nullableCompositeType(type)) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
-    private static boolean nullableCompositeType(IAType type) {
-        if (type.getTypeTag() == ATypeTag.UNION || type.getTypeTag() == ATypeTag.NULL
-                || type.getTypeTag() == ATypeTag.ANY) {
-            return true;
-        } else if (type.getTypeTag() == ATypeTag.RECORD) {
-            ARecordType recordType = (ARecordType) type;
-            IAType[] fieldTypes = recordType.getFieldTypes();
-            for (IAType fieldType : fieldTypes) {
-                boolean nullable = nullableCompositeType(fieldType);
-                if (nullable) {
-                    return true;
-                }
-            }
-            return false;
-        } else if (type.getTypeTag() == ATypeTag.UNORDEREDLIST || type.getTypeTag() == ATypeTag.ORDEREDLIST) {
-            AbstractCollectionType collectionType = (AbstractCollectionType) type;
-            IAType itemType = collectionType.getItemType();
-            boolean nullable = nullableCompositeType(itemType);
-            if (nullable) {
-                return true;
-            } else {
-                return false;
-            }
-        } else {
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
index 311c54f..7f47961 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
@@ -19,23 +19,19 @@
 
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ABinaryTypeComputer implements IResultTypeComputer {
+public class ABinaryTypeComputer extends AbstractResultTypeComputer {
     public static final ABinaryTypeComputer INSTANCE = new ABinaryTypeComputer();
 
     private ABinaryTypeComputer() {
 
     }
 
-    @Override public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ABINARY;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
index a855fd2..89ffd4c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ABooleanTypeComputer implements IResultTypeComputer {
+public class ABooleanTypeComputer extends AbstractResultTypeComputer {
 
     public static final ABooleanTypeComputer INSTANCE = new ABooleanTypeComputer();
 
@@ -34,8 +30,7 @@ public class ABooleanTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ABOOLEAN;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ACircleTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ACircleTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ACircleTypeComputer.java
index ab178df..e81d2f4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ACircleTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ACircleTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ACircleTypeComputer implements IResultTypeComputer {
+public class ACircleTypeComputer extends AbstractResultTypeComputer {
 
     public static final ACircleTypeComputer INSTANCE = new ACircleTypeComputer();
 
@@ -34,8 +30,7 @@ public class ACircleTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ACIRCLE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
index 319de99..2f4d0d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ADateTimeTypeComputer implements IResultTypeComputer {
+public class ADateTimeTypeComputer extends AbstractResultTypeComputer {
 
     public static final ADateTimeTypeComputer INSTANCE = new ADateTimeTypeComputer();
 
@@ -34,8 +30,7 @@ public class ADateTimeTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ADATETIME;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
index 8be058c..dd892cd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ADateTypeComputer implements IResultTypeComputer {
+public class ADateTypeComputer extends AbstractResultTypeComputer {
 
     public static final ADateTypeComputer INSTANCE = new ADateTypeComputer();
 
@@ -34,8 +30,7 @@ public class ADateTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ADATE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
new file mode 100644
index 0000000..6139b0d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class ADayTimeDurationTypeComputer extends AbstractResultTypeComputer {
+
+    public static final ADayTimeDurationTypeComputer INSTANCE = new ADayTimeDurationTypeComputer();
+
+    private ADayTimeDurationTypeComputer() {
+
+    }
+
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
+        return BuiltinType.ADAYTIMEDURATION;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
index 9e468bd..eb3de27 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ADoubleTypeComputer implements IResultTypeComputer {
+public class ADoubleTypeComputer extends AbstractResultTypeComputer {
 
     public static final ADoubleTypeComputer INSTANCE = new ADoubleTypeComputer();
 
@@ -34,8 +30,7 @@ public class ADoubleTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ADOUBLE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
new file mode 100644
index 0000000..9021284
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class ADurationTypeComputer extends AbstractResultTypeComputer {
+
+    public static final ADurationTypeComputer INSTANCE = new ADurationTypeComputer();
+
+    private ADurationTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
+        return BuiltinType.ADURATION;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
index 052a5a9..d2fd672 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
@@ -18,14 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AFloatTypeComputer implements IResultTypeComputer {
+public class AFloatTypeComputer extends AbstractResultTypeComputer {
 
     public static final AFloatTypeComputer INSTANCE = new AFloatTypeComputer();
 
@@ -33,8 +30,7 @@ public class AFloatTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.AFLOAT;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
new file mode 100644
index 0000000..ed9d785
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class AInt16TypeComputer extends AbstractResultTypeComputer {
+
+    public static final AInt16TypeComputer INSTANCE = new AInt16TypeComputer();
+
+    private AInt16TypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
+        return BuiltinType.AINT16;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
index 782e246..f0c8f89 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AInt32TypeComputer implements IResultTypeComputer {
+public class AInt32TypeComputer extends AbstractResultTypeComputer {
 
     public static final AInt32TypeComputer INSTANCE = new AInt32TypeComputer();
 
@@ -34,8 +30,7 @@ public class AInt32TypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.AINT32;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
index c6468ca..7c43d67 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AInt64TypeComputer implements IResultTypeComputer {
+public class AInt64TypeComputer extends AbstractResultTypeComputer {
 
     public static final AInt64TypeComputer INSTANCE = new AInt64TypeComputer();
 
@@ -34,8 +30,7 @@ public class AInt64TypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.AINT64;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
new file mode 100644
index 0000000..021e514
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class AInt8TypeComputer extends AbstractResultTypeComputer {
+
+    public static final AInt8TypeComputer INSTANCE = new AInt8TypeComputer();
+
+    private AInt8TypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
+        return BuiltinType.AINT8;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
index f6e3183..93d5a9a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
@@ -18,27 +18,19 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AIntervalTypeComputer implements IResultTypeComputer {
+public class AIntervalTypeComputer extends AbstractResultTypeComputer {
 
     public static final AIntervalTypeComputer INSTANCE = new AIntervalTypeComputer();
 
     private AIntervalTypeComputer() {
     }
 
-    /* (non-Javadoc)
-     * @see org.apache.asterix.om.typecomputer.base.IResultTypeComputer#computeType(org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression, org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment, org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider)
-     */
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.AINTERVAL;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ALineTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ALineTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ALineTypeComputer.java
index 4250c3a..6c93088 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ALineTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ALineTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ALineTypeComputer implements IResultTypeComputer {
+public class ALineTypeComputer extends AbstractResultTypeComputer {
 
     public static final ALineTypeComputer INSTANCE = new ALineTypeComputer();
 
@@ -34,8 +30,7 @@ public class ALineTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ALINE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AMissingTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AMissingTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AMissingTypeComputer.java
new file mode 100644
index 0000000..bf262c3
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AMissingTypeComputer.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class AMissingTypeComputer implements IResultTypeComputer {
+
+    public static final AMissingTypeComputer INSTANCE = new AMissingTypeComputer();
+
+    private AMissingTypeComputer() {
+    }
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        return BuiltinType.AMISSING;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ANullTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ANullTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ANullTypeComputer.java
deleted file mode 100644
index 7803760..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ANullTypeComputer.java
+++ /dev/null
@@ -1,42 +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.typecomputer.base.IResultTypeComputer;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public class ANullTypeComputer implements IResultTypeComputer {
-
-    public static final ANullTypeComputer INSTANCE = new ANullTypeComputer();
-
-    private ANullTypeComputer() {
-    }
-
-    @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-        return BuiltinType.ANULL;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APoint3DTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APoint3DTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APoint3DTypeComputer.java
index cc7be44..74dc67b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APoint3DTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APoint3DTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class APoint3DTypeComputer implements IResultTypeComputer {
+public class APoint3DTypeComputer extends AbstractResultTypeComputer {
 
     public static final APoint3DTypeComputer INSTANCE = new APoint3DTypeComputer();
 
@@ -34,8 +30,7 @@ public class APoint3DTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.APOINT3D;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APointTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APointTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APointTypeComputer.java
index 133c845..d6cd6f7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APointTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APointTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class APointTypeComputer implements IResultTypeComputer {
+public class APointTypeComputer extends AbstractResultTypeComputer {
 
     public static final APointTypeComputer INSTANCE = new APointTypeComputer();
 
@@ -34,8 +30,7 @@ public class APointTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.APOINT;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APolygonTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APolygonTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APolygonTypeComputer.java
index 69a9640..3c7e383 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APolygonTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/APolygonTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class APolygonTypeComputer implements IResultTypeComputer {
+public class APolygonTypeComputer extends AbstractResultTypeComputer {
 
     public static final APolygonTypeComputer INSTANCE = new APolygonTypeComputer();
 
@@ -34,8 +30,7 @@ public class APolygonTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.APOLYGON;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ARectangleTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ARectangleTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ARectangleTypeComputer.java
index 03c80b2..f308b81 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ARectangleTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ARectangleTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ARectangleTypeComputer implements IResultTypeComputer {
+public class ARectangleTypeComputer extends AbstractResultTypeComputer {
 
     public static final ARectangleTypeComputer INSTANCE = new ARectangleTypeComputer();
 
@@ -34,8 +30,7 @@ public class ARectangleTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ARECTANGLE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
index 82c22b8..f900b77 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AStringTypeComputer implements IResultTypeComputer {
+public class AStringTypeComputer extends AbstractResultTypeComputer {
 
     public static final AStringTypeComputer INSTANCE = new AStringTypeComputer();
 
@@ -34,8 +30,7 @@ public class AStringTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ASTRING;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATemporalInstanceTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATemporalInstanceTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATemporalInstanceTypeComputer.java
new file mode 100644
index 0000000..7d20bd3
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATemporalInstanceTypeComputer.java
@@ -0,0 +1,52 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class ATemporalInstanceTypeComputer implements IResultTypeComputer {
+
+    public static final ATemporalInstanceTypeComputer INSTANCE = new ATemporalInstanceTypeComputer();
+
+    private ATemporalInstanceTypeComputer() {
+    }
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        List<IAType> unionList = new ArrayList<>();
+        unionList.add(BuiltinType.ADATE);
+        unionList.add(BuiltinType.ATIME);
+        unionList.add(BuiltinType.ADATETIME);
+        unionList.add(BuiltinType.AMISSING);
+        unionList.add(BuiltinType.ANULL);
+        return new AUnionType(unionList, "OptionalTemporalInstance");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
index 1897244..a6950e1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class ATimeTypeComputer implements IResultTypeComputer {
+public class ATimeTypeComputer extends AbstractResultTypeComputer {
 
     public static final ATimeTypeComputer INSTANCE = new ATimeTypeComputer();
 
@@ -34,8 +30,7 @@ public class ATimeTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.ATIME;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
index d5d324c..0a0c923 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
@@ -19,15 +19,11 @@
 
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class AUUIDTypeComputer implements IResultTypeComputer {
+public class AUUIDTypeComputer extends AbstractResultTypeComputer {
 
     public static final AUUIDTypeComputer INSTANCE = new AUUIDTypeComputer();
 
@@ -35,8 +31,7 @@ public class AUUIDTypeComputer implements IResultTypeComputer {
     }
 
     @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+    protected IAType getResultType(IAType... strippedInputTypes) {
         return BuiltinType.AUUID;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
new file mode 100644
index 0000000..6360f11
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class AYearMonthDurationTypeComputer extends AbstractResultTypeComputer {
+
+    public static final AYearMonthDurationTypeComputer INSTANCE = new AYearMonthDurationTypeComputer();
+
+    private AYearMonthDurationTypeComputer() {
+
+    }
+
+    @Override
+    protected IAType getResultType(IAType... strippedInputTypes) {
+        return BuiltinType.AYEARMONTHDURATION;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java
deleted file mode 100644
index 522218e..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java
+++ /dev/null
@@ -1,90 +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.typecomputer.base.IResultTypeComputer;
-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.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-/**
- * @author Xiaoyu Ma
- */
-abstract public class AbstractBinaryStringTypeComputer implements IResultTypeComputer {
-
-    @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
-        if (fce.getArguments().size() < 2) {
-            throw new AlgebricksException("Wrong Argument Number.");
-        }
-        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
-        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
-        IAType t0 = (IAType) env.getType(arg0);
-        IAType t1 = (IAType) env.getType(arg1);
-
-        boolean nullable = false;
-        ATypeTag tag0, tag1;
-        if (NonTaggedFormatUtil.isOptional(t0)) {
-            tag0 = ((AUnionType) t0).getNullableType().getTypeTag();
-            nullable = true;
-        } else {
-            tag0 = t0.getTypeTag();
-        }
-
-        if (NonTaggedFormatUtil.isOptional(t1)) {
-            tag1 = ((AUnionType) t1).getNullableType().getTypeTag();
-            nullable = true;
-        } else {
-            tag1 = t1.getTypeTag();
-        }
-
-        if (tag0 == ATypeTag.NULL && tag1 == ATypeTag.NULL) {
-            return BuiltinType.ANULL;
-        }
-
-        if (tag0 == ATypeTag.NULL || tag1 == ATypeTag.NULL) {
-            nullable = true;
-        }
-
-        // Allow substring to work with ANY types, i.e., types that are unknown at compile time.
-        if (tag0 == ATypeTag.ANY || tag1 == ATypeTag.ANY) {
-            return BuiltinType.ANY;
-        }
-
-        if (tag0 != ATypeTag.NULL && tag0 != ATypeTag.STRING) {
-            throw new AlgebricksException("First argument should be String Type, but it is " + tag0);
-        }
-
-        if (tag1 != ATypeTag.NULL && tag1 != ATypeTag.STRING) {
-            throw new AlgebricksException("Second argument should be String Type, but it is " + tag1);
-        }
-        return getResultType(t0, t1, nullable);
-    }
-
-    public abstract IAType getResultType(IAType t0, IAType t1, boolean nullable);
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java
deleted file mode 100644
index 2abdf08..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java
+++ /dev/null
@@ -1,66 +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.typecomputer.base.IResultTypeComputer;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-/**
- * @author Xiaoyu Ma
- */
-public abstract class AbstractQuadStringTypeComputer implements IResultTypeComputer {
-
-    @Override
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
-        if (fce.getArguments().size() < 4)
-            throw new AlgebricksException("Wrong Argument Number.");
-        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
-        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
-        ILogicalExpression arg2 = fce.getArguments().get(2).getValue();
-        ILogicalExpression arg3 = fce.getArguments().get(3).getValue();
-        IAType t0, t1, t2, t3;
-        try {
-            t0 = (IAType) env.getType(arg0);
-            t1 = (IAType) env.getType(arg1);
-            t2 = (IAType) env.getType(arg2);
-            t3 = (IAType) env.getType(arg3);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
-        if ((t0.getTypeTag() != ATypeTag.NULL && t0.getTypeTag() != ATypeTag.STRING)
-                || (t1.getTypeTag() != ATypeTag.NULL && t1.getTypeTag() != ATypeTag.STRING)
-                || (t2.getTypeTag() != ATypeTag.NULL && t2.getTypeTag() != ATypeTag.STRING)
-                || (t3.getTypeTag() != ATypeTag.NULL && t3.getTypeTag() != ATypeTag.STRING)) {
-            throw new NotImplementedException("Expects String Type.");
-        }
-
-        return getResultType(t0, t1, t2, t3);
-    }
-
-    public abstract IAType getResultType(IAType t0, IAType t1, IAType t2, IAType t3);
-}