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:27 UTC

[09/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-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
index 01abb1b..7150517 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -23,6 +23,7 @@ import java.io.DataOutput;
 import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -80,6 +81,9 @@ public class AndDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ANULL);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AMissing> missingSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.AMISSING);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
@@ -88,25 +92,40 @@ public class AndDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                             int n = args.length;
                             boolean res = true;
                             boolean metNull = false;
+                            boolean metMissing = false;
                             for (int i = 0; i < n; i++) {
                                 evals[i].evaluate(tuple, argPtr);
                                 byte[] bytes = argPtr.getByteArray();
                                 int offset = argPtr.getStartOffset();
-
+                                boolean isNull = false;
+                                boolean isMissing = false;
+                                if (bytes[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+                                    isMissing = true;
+                                    metMissing = true;
+                                }
                                 if (bytes[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+                                    isNull = true;
                                     metNull = true;
+                                }
+                                if (isMissing || isNull) {
                                     continue;
                                 }
                                 boolean argResult = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
-                                res = res && argResult;
-                            }
-                            if (metNull) {
-                                if (!res) {
-                                    ABoolean aResult = ABoolean.FALSE;
-                                    booleanSerde.serialize(aResult, out);
-                                } else {
-                                    nullSerde.serialize(ANull.NULL, out);
+                                if (argResult == false) {
+                                    // anything AND FALSE = FALSE
+                                    booleanSerde.serialize(ABoolean.FALSE, out);
+                                    result.set(resultStorage);
+                                    return;
                                 }
+                                res &= argResult;
+                            }
+                            if (metMissing) {
+                                // MISSING AND NULL = MISSING
+                                // MISSING AND TRUE = MISSING
+                                missingSerde.serialize(AMissing.MISSING, out);
+                            } else if (metNull) {
+                                // NULL AND TRUE = NULL
+                                nullSerde.serialize(ANull.NULL, out);
                             } else {
                                 ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
                                 booleanSerde.serialize(aResult, out);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index 546eebd..16cec14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -24,13 +24,10 @@ import java.io.IOException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 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.util.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -39,7 +36,6 @@ 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;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -86,9 +82,6 @@ public class AnyCollectionMemberDescriptor extends AbstractScalarFunctionDynamic
                 private DataOutput out = resultStorage.getDataOutput();
                 private IPointable inputArgList = new VoidPointable();
                 private IScalarEvaluator evalList = listEvalFactory.createScalarEvaluator(ctx);
-                @SuppressWarnings("unchecked")
-                private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                        .getSerializerDeserializer(BuiltinType.ANULL);
                 private int itemOffset;
                 private int itemLength;
 
@@ -101,12 +94,6 @@ public class AnyCollectionMemberDescriptor extends AbstractScalarFunctionDynamic
                         byte[] serList = inputArgList.getByteArray();
                         int offset = inputArgList.getStartOffset();
 
-                        if (serList[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                            nullSerde.serialize(ANull.NULL, out);
-                            result.set(resultStorage);
-                            return;
-                        }
-
                         if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
                             throw new AlgebricksException(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER.getName()
@@ -116,14 +103,14 @@ public class AnyCollectionMemberDescriptor extends AbstractScalarFunctionDynamic
 
                         if (serList[offset] == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
                             if (AOrderedListSerializerDeserializer.getNumberOfItems(serList, offset) == 0) {
-                                out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                                out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                                 result.set(resultStorage);
                                 return;
                             }
                             itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serList, offset, 0);
                         } else {
                             if (AUnorderedListSerializerDeserializer.getNumberOfItems(serList, offset) == 0) {
-                                out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                                out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                                 result.set(resultStorage);
                                 return;
                             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
new file mode 100644
index 0000000..abd0d3a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
@@ -0,0 +1,86 @@
+/*
+ * 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.evaluators.functions;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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;
+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;
+
+/**
+ * This runtime function checks if the input is null.
+ * If the input is not null, just return it directly;
+ * Otherwise, throw a runtime exception.
+ */
+public class CheckUnknownDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new CheckUnknownDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+                return new IScalarEvaluator() {
+                    private IPointable inputArg = new VoidPointable();
+                    private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+                    private String errorMessage = AsterixBuiltinFunctions.CHECK_UNKNOWN
+                            + ": the input value cannot be NULL.";
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+                        eval.evaluate(tuple, inputArg);
+                        byte[] data = inputArg.getByteArray();
+                        int offset = inputArg.getStartOffset();
+
+                        if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
+                                || data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+                            throw new AlgebricksException(errorMessage);
+                        }
+                        result.set(inputArg);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.CHECK_UNKNOWN;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index b65c5c0..6d37152 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -22,10 +22,8 @@ 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.ANullSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutablePoint;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.APoint;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -86,10 +84,8 @@ public class CreatePointDescriptor extends AbstractScalarFunctionDynamicDescript
                         int offset1 = inputArg1.getStartOffset();
 
                         // type-check: (double, double)
-                        if ((bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG
-                                && bytes0[offset0] != ATypeTag.SERIALIZED_NULL_TYPE_TAG)
-                                || (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG
-                                        && bytes1[offset1] != ATypeTag.SERIALIZED_NULL_TYPE_TAG)) {
+                        if (bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG
+                                || bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
                             throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_POINT.getName()
                                     + ": expects input type: (DOUBLE, DOUBLE) but got ("
                                     + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
@@ -98,14 +94,9 @@ public class CreatePointDescriptor extends AbstractScalarFunctionDynamicDescript
 
                         resultStorage.reset();
                         try {
-                            if (bytes0[offset0] == ATypeTag.SERIALIZED_NULL_TYPE_TAG
-                                    || bytes1[offset1] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
-                            } else {
-                                aPoint.setValue(ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1),
-                                        ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1));
-                                pointSerde.serialize(aPoint, out);
-                            }
+                            aPoint.setValue(ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1),
+                                    ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1));
+                            pointSerde.serialize(aPoint, out);
                         } catch (IOException e1) {
                             throw new AlgebricksException(e1);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
index 174deb1..dec8829 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
@@ -76,9 +76,9 @@ public class DeepEqualityDescriptor extends AbstractScalarFunctionDynamicDescrip
                 final DataOutput out = resultStorage.getDataOutput();
                 final IScalarEvaluator evalLeft = evalFactoryLeft.createScalarEvaluator(ctx);
                 final IScalarEvaluator evalRight = evalFactoryRight.createScalarEvaluator(ctx);
-                final DeepEqualAssessor deepEqualAssessor = new DeepEqualAssessor();
 
                 return new IScalarEvaluator() {
+                    private final DeepEqualAssessor deepEqualAssessor = new DeepEqualAssessor();
                     private final PointableAllocator allocator = new PointableAllocator();
                     private final IVisitablePointable pointableLeft = allocator.allocateFieldValue(inputTypeLeft);
                     private final IVisitablePointable pointableRight = allocator.allocateFieldValue(inputTypeRight);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java
deleted file mode 100644
index 01ddb55..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java
+++ /dev/null
@@ -1,158 +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.runtime.evaluators.functions;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-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.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-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;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-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;
-
-/**
- * Checks whether a list with an edit distance threshold can be filtered with a lower bounding on the number of common list elements.
- * This function returns 'true' if the lower bound on the number of common elements is positive, 'false' otherwise.
- * For example, this function is used during an indexed nested-loop join based on edit distance. We partition the tuples of the probing
- * dataset into those that are filterable and those that are not. Those that are filterable are forwarded to the index. The others are
- * are fed into a (non indexed) nested-loop join.
- */
-public class EditDistanceListIsFilterable extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new EditDistanceListIsFilterable();
-        }
-    };
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
-            throws AlgebricksException {
-        return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
-                return new EditDistanceListIsFilterableEvaluator(args, ctx);
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE;
-    }
-
-    private static class EditDistanceListIsFilterableEvaluator implements IScalarEvaluator {
-
-        protected final IPointable argPtr = new VoidPointable();
-        protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-        protected final DataOutput output = resultStorage.getDataOutput();
-
-        protected final IScalarEvaluator listEval;
-        protected final IScalarEvaluator edThreshEval;
-
-        @SuppressWarnings("unchecked")
-        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-
-        public EditDistanceListIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-                throws AlgebricksException {
-            listEval = args[0].createScalarEvaluator(context);
-            edThreshEval = args[1].createScalarEvaluator(context);
-        }
-
-        @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            ATypeTag typeTag = null;
-
-            // Check type and compute string length.
-            listEval.evaluate(tuple, argPtr);
-            byte[] bytes = argPtr.getByteArray();
-            int offset = argPtr.getStartOffset();
-
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
-            long listLen = 0;
-            switch (typeTag) {
-                case UNORDEREDLIST: {
-                    listLen = AUnorderedListSerializerDeserializer.getNumberOfItems(bytes, offset);
-                    break;
-                }
-                case ORDEREDLIST: {
-                    listLen = AOrderedListSerializerDeserializer.getNumberOfItems(bytes, offset);
-                    break;
-                }
-                default: {
-                    throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName()
-                            + ": expects input type ORDEREDLIST or UNORDEREDLIST as the first argument, but got "
-                            + typeTag + ".");
-                }
-            }
-
-            // Check type and extract edit-distance threshold.
-            edThreshEval.evaluate(tuple, argPtr);
-            bytes = argPtr.getByteArray();
-            offset = argPtr.getStartOffset();
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
-            long edThresh;
-
-            try {
-                edThresh = ATypeHierarchy.getIntegerValue(bytes, offset);
-            } catch (HyracksDataException e1) {
-                throw new AlgebricksException(e1);
-            }
-
-            // Compute result.
-            long lowerBound = listLen - edThresh;
-            try {
-                if (lowerBound <= 0) {
-                    booleanSerde.serialize(ABoolean.FALSE, output);
-                } else {
-                    booleanSerde.serialize(ABoolean.TRUE, output);
-                }
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
-            }
-            result.set(resultStorage);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
new file mode 100644
index 0000000..a4c4311
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
@@ -0,0 +1,156 @@
+/*
+ * 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.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+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.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+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;
+
+/**
+ * Checks whether a list with an edit distance threshold can be filtered with a lower bounding on the number
+ * of common list elements. This function returns 'true' if the lower bound on the number of common elements
+ * is positive, 'false' otherwise. For example, this function is used during an indexed nested-loop join based
+ * on edit distance. We partition the tuples of the probing dataset into those that are filterable and those
+ * that are not. Those that are filterable are forwarded to the index. The others are are fed into a (non
+ * indexed) nested-loop join.
+ */
+public class EditDistanceListIsFilterableDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new EditDistanceListIsFilterableDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+                return new EditDistanceListIsFilterableEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE;
+    }
+
+    private static class EditDistanceListIsFilterableEvaluator implements IScalarEvaluator {
+
+        protected final IPointable listPtr = new VoidPointable();
+        protected final IPointable edThreshPtr = new VoidPointable();
+        protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+        protected final DataOutput output = resultStorage.getDataOutput();
+
+        protected final IScalarEvaluator listEval;
+        protected final IScalarEvaluator edThreshEval;
+
+        @SuppressWarnings("unchecked")
+        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+        public EditDistanceListIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+                throws AlgebricksException {
+            listEval = args[0].createScalarEvaluator(context);
+            edThreshEval = args[1].createScalarEvaluator(context);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+            resultStorage.reset();
+
+            listEval.evaluate(tuple, listPtr);
+            edThreshEval.evaluate(tuple, edThreshPtr);
+
+            // Check type and compute string length.
+            byte[] bytes = listPtr.getByteArray();
+            int offset = listPtr.getStartOffset();
+
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+            long listLen;
+            switch (typeTag) {
+                case UNORDEREDLIST:
+                    listLen = AUnorderedListSerializerDeserializer.getNumberOfItems(bytes, offset);
+                    break;
+                case ORDEREDLIST:
+                    listLen = AOrderedListSerializerDeserializer.getNumberOfItems(bytes, offset);
+                    break;
+                default:
+                    throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName()
+                            + ": expects input type ORDEREDLIST or UNORDEREDLIST as the first argument, but got "
+                            + typeTag + ".");
+            }
+
+            // Check type and extract edit-distance threshold.
+            bytes = edThreshPtr.getByteArray();
+            offset = edThreshPtr.getStartOffset();
+            long edThresh;
+
+            try {
+                edThresh = ATypeHierarchy.getIntegerValue(bytes, offset);
+            } catch (HyracksDataException e1) {
+                throw new AlgebricksException(e1);
+            }
+
+            // Compute result.
+            long lowerBound = listLen - edThresh;
+            try {
+                if (lowerBound <= 0) {
+                    booleanSerde.serialize(ABoolean.FALSE, output);
+                } else {
+                    booleanSerde.serialize(ABoolean.TRUE, output);
+                }
+            } catch (IOException e) {
+                throw new AlgebricksException(e);
+            }
+            result.set(resultStorage);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java
deleted file mode 100644
index a707e46..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java
+++ /dev/null
@@ -1,171 +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.runtime.evaluators.functions;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-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.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-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;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.BooleanPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-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;
-
-/**
- * Checks whether a string with an edit distance threshold can be filtered with a lower bounding on number of common grams.
- * This function returns 'true' if the lower bound on the number of common grams is positive, 'false' otherwise.
- * For example, this function is used during an indexed nested-loop join based on edit distance. We partition the tuples of the probing
- * dataset into those that are filterable and those that are not. Those that are filterable are forwarded to the index. The others are
- * are fed into a (non indexed) nested-loop join.
- */
-public class EditDistanceStringIsFilterable extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new EditDistanceStringIsFilterable();
-        }
-    };
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
-            throws AlgebricksException {
-        return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
-                return new EditDistanceStringIsFilterableEvaluator(args, ctx);
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE;
-    }
-
-    private static class EditDistanceStringIsFilterableEvaluator implements IScalarEvaluator {
-
-        protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-        protected final DataOutput output = resultStorage.getDataOutput();
-        protected final IPointable argPtr = new VoidPointable();
-
-        protected final IScalarEvaluator stringEval;
-        protected final IScalarEvaluator edThreshEval;
-        protected final IScalarEvaluator gramLenEval;
-        protected final IScalarEvaluator usePrePostEval;
-
-        @SuppressWarnings("unchecked")
-        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-
-        private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-        public EditDistanceStringIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-                throws AlgebricksException {
-            stringEval = args[0].createScalarEvaluator(context);
-            edThreshEval = args[1].createScalarEvaluator(context);
-            gramLenEval = args[2].createScalarEvaluator(context);
-            usePrePostEval = args[3].createScalarEvaluator(context);
-        }
-
-        @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            ATypeTag typeTag = null;
-
-            // Check type and compute string length.
-            stringEval.evaluate(tuple, argPtr);
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr.getByteArray()[argPtr.getStartOffset()]);
-            if (!typeTag.equals(ATypeTag.STRING)) {
-                throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
-                        + ": expects input type STRING as first argument, but got " + typeTag + ".");
-            }
-
-            utf8Ptr.set(argPtr.getByteArray(), argPtr.getStartOffset() + 1, argPtr.getLength());
-            int strLen = utf8Ptr.getStringLength();
-
-            // Check type and extract edit-distance threshold.
-            edThreshEval.evaluate(tuple, argPtr);
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr.getByteArray()[argPtr.getStartOffset()]);
-
-            long edThresh = 0;
-
-            try {
-                edThresh = ATypeHierarchy.getIntegerValue(argPtr.getByteArray(), argPtr.getStartOffset());
-            } catch (HyracksDataException e1) {
-                throw new AlgebricksException(e1);
-            }
-
-            // Check type and extract gram length.
-            gramLenEval.evaluate(tuple, argPtr);
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr.getByteArray()[argPtr.getStartOffset()]);
-
-            long gramLen = 0;
-            try {
-                gramLen = ATypeHierarchy.getIntegerValue(argPtr.getByteArray(), argPtr.getStartOffset());
-            } catch (HyracksDataException e1) {
-                throw new AlgebricksException(e1);
-            }
-
-            // Check type and extract usePrePost flag.
-            usePrePostEval.evaluate(tuple, argPtr);
-            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr.getByteArray()[argPtr.getStartOffset()]);
-            if (!typeTag.equals(ATypeTag.BOOLEAN)) {
-                throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
-                        + ": expects input type BOOLEAN as fourth argument, but got " + typeTag + ".");
-            }
-            boolean usePrePost = BooleanPointable.getBoolean(argPtr.getByteArray(), argPtr.getStartOffset() + 1);
-
-            // Compute result.
-            long numGrams = (usePrePost) ? strLen + gramLen - 1 : strLen - gramLen + 1;
-            long lowerBound = numGrams - edThresh * gramLen;
-            try {
-                if (lowerBound <= 0 || strLen == 0) {
-                    booleanSerde.serialize(ABoolean.FALSE, output);
-                } else {
-                    booleanSerde.serialize(ABoolean.TRUE, output);
-                }
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
-            }
-            result.set(resultStorage);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
new file mode 100644
index 0000000..8944453
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.evaluators.functions;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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;
+
+/**
+ * Checks whether a string with an edit distance threshold can be filtered with a lower bounding
+ * on number of common grams. This function returns 'true' if the lower bound on the number of
+ * common grams is positive, 'false' otherwise. For example, this function is used during an indexed
+ * nested-loop join based on edit distance. We partition the tuples of the probing dataset into those
+ * that are filterable and those that are not. Those that are filterable are forwarded to the index.
+ * The others are fed into a (non indexed) nested-loop join.
+ */
+public class EditDistanceStringIsFilterableDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new EditDistanceStringIsFilterableDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+                return new EditDistanceStringIsFilterableEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
new file mode 100644
index 0000000..595a85a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
@@ -0,0 +1,132 @@
+/*
+ * 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.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.BooleanPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+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;
+
+public class EditDistanceStringIsFilterableEvaluator implements IScalarEvaluator {
+
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput output = resultStorage.getDataOutput();
+    protected final IPointable stringPtr = new VoidPointable();
+    protected final IPointable edThreshPtr = new VoidPointable();
+    protected final IPointable gramLenPtr = new VoidPointable();
+    protected final IPointable usePrePostPtr = new VoidPointable();
+
+    protected final IScalarEvaluator stringEval;
+    protected final IScalarEvaluator edThreshEval;
+    protected final IScalarEvaluator gramLenEval;
+    protected final IScalarEvaluator usePrePostEval;
+
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    public EditDistanceStringIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+            throws AlgebricksException {
+        stringEval = args[0].createScalarEvaluator(context);
+        edThreshEval = args[1].createScalarEvaluator(context);
+        gramLenEval = args[2].createScalarEvaluator(context);
+        usePrePostEval = args[3].createScalarEvaluator(context);
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+        resultStorage.reset();
+
+        stringEval.evaluate(tuple, stringPtr);
+        edThreshEval.evaluate(tuple, edThreshPtr);
+        gramLenEval.evaluate(tuple, gramLenPtr);
+        usePrePostEval.evaluate(tuple, usePrePostPtr);
+
+        // Check type and compute string length.
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(stringPtr.getByteArray()[stringPtr.getStartOffset()]);
+        if (!typeTag.equals(ATypeTag.STRING)) {
+            throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
+                    + ": expects input type STRING as first argument, but got " + typeTag + ".");
+        }
+        utf8Ptr.set(stringPtr.getByteArray(), stringPtr.getStartOffset() + 1, stringPtr.getLength());
+        int strLen = utf8Ptr.getStringLength();
+
+        // Check type and extract edit-distance threshold.
+        long edThresh = 0;
+        try {
+            edThresh = ATypeHierarchy.getIntegerValue(edThreshPtr.getByteArray(), edThreshPtr.getStartOffset());
+        } catch (HyracksDataException e1) {
+            throw new AlgebricksException(e1);
+        }
+
+        // Check type and extract gram length.
+        long gramLen = 0;
+        try {
+            gramLen = ATypeHierarchy.getIntegerValue(gramLenPtr.getByteArray(), gramLenPtr.getStartOffset());
+        } catch (HyracksDataException e1) {
+            throw new AlgebricksException(e1);
+        }
+
+        // Check type and extract usePrePost flag.
+        typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(usePrePostPtr.getByteArray()[usePrePostPtr.getStartOffset()]);
+        if (!typeTag.equals(ATypeTag.BOOLEAN)) {
+            throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
+                    + ": expects input type BOOLEAN as fourth argument, but got " + typeTag + ".");
+        }
+        boolean usePrePost = BooleanPointable.getBoolean(usePrePostPtr.getByteArray(),
+                usePrePostPtr.getStartOffset() + 1);
+
+        // Compute result.
+        long numGrams = usePrePost ? strLen + gramLen - 1 : strLen - gramLen + 1;
+        long lowerBound = numGrams - edThresh * gramLen;
+        try {
+            if (lowerBound <= 0 || strLen == 0) {
+                booleanSerde.serialize(ABoolean.FALSE, output);
+            } else {
+                booleanSerde.serialize(ABoolean.TRUE, output);
+            }
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+        result.set(resultStorage);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
deleted file mode 100644
index 0bfa5a8..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
+++ /dev/null
@@ -1,77 +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.runtime.evaluators.functions;
-
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-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;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-public class EmbedTypeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new EmbedTypeDescriptor();
-        }
-    };
-
-    @SuppressWarnings("unused")
-    private IAType fieldType;
-
-    public void reset(IAType fieldType) {
-        this.fieldType = fieldType;
-    }
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new IScalarEvaluatorFactory() {
-
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
-                // TODO Auto-generated method stub
-                return new IScalarEvaluator() {
-
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-                        // TODO Auto-generated method stub
-
-                    }
-                };
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.EMBED_TYPE;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
deleted file mode 100644
index 5c9d8ef..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
+++ /dev/null
@@ -1,91 +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.runtime.evaluators.functions;
-
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.pointables.PointableAllocator;
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-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;
-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;
-
-public class FlowRecordDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new FlowRecordDescriptor();
-        }
-    };
-
-    private static final long serialVersionUID = 1L;
-    private ARecordType inputType;
-
-    public void reset(ARecordType inputType) {
-        this.inputType = inputType;
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.FLOW_RECORD;
-    }
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        final IScalarEvaluatorFactory recordEvalFactory = args[0];
-
-        return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
-                final IPointable recordPtr = new VoidPointable();
-                final IScalarEvaluator recEvaluator = recordEvalFactory.createScalarEvaluator(ctx);
-
-                return new IScalarEvaluator() {
-                    // pointable allocator
-                    private PointableAllocator allocator = new PointableAllocator();
-                    final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
-
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-                        try {
-                            recEvaluator.evaluate(tuple, recordPtr);
-                            recAccessor.set(recordPtr);
-                            result.set(recAccessor);
-                        } catch (Exception ioe) {
-                            throw new AlgebricksException(ioe);
-                        }
-                    }
-                };
-            }
-        };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
deleted file mode 100644
index eaa6028..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
+++ /dev/null
@@ -1,49 +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.runtime.evaluators.functions;
-
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-
-public class FuzzyEqDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new FuzzyEqDescriptor();
-        }
-    };
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.FUZZY_EQ;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index 020336f..88b8a51 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -85,7 +85,7 @@ public class GetItemDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                 private IPointable inputArgIdx = new VoidPointable();
                 private IScalarEvaluator evalList = listEvalFactory.createScalarEvaluator(ctx);
                 private IScalarEvaluator evalIdx = indexEvalFactory.createScalarEvaluator(ctx);
-                private byte[] nullBytes = new byte[] { ATypeTag.SERIALIZED_NULL_TYPE_TAG };
+                private byte[] missingBytes = new byte[] { ATypeTag.SERIALIZED_MISSING_TYPE_TAG };
                 private int itemIndex;
                 private int itemOffset;
                 private int itemLength;
@@ -102,11 +102,6 @@ public class GetItemDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                         byte[] indexBytes = inputArgIdx.getByteArray();
                         int indexOffset = inputArgIdx.getStartOffset();
 
-                        if (serOrderedList[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                            result.set(nullBytes, 0, 1);
-                            return;
-                        }
-
                         if (serOrderedList[offset] == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
                             itemIndex = ATypeHierarchy.getIntegerValue(indexBytes, indexOffset);
                         } else {
@@ -118,14 +113,12 @@ public class GetItemDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                                     + ").");
                         }
 
-                        if (itemIndex >= AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList, offset)) {
-                            result.set(nullBytes, 0, 1);
+                        if (itemIndex < 0 || itemIndex >= AOrderedListSerializerDeserializer
+                                .getNumberOfItems(serOrderedList, offset)) {
+                            // Out-of-bound index access should return MISSING.
+                            result.set(missingBytes, 0, 1);
                             return;
                         }
-                        if (itemIndex < 0) {
-                            throw new AlgebricksException(
-                                    AsterixBuiltinFunctions.GET_ITEM.getName() + ": item index cannot be negative!");
-                        }
 
                         itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + 1]);
                         if (itemTag == ATypeTag.ANY) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
new file mode 100644
index 0000000..45c3f6d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.evaluators.functions;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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;
+
+public class IsMissingDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new IsMissingDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+                final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+                return new AbstractTypeCheckEvaluator(eval) {
+
+                    @Override
+                    protected boolean isMatch(byte typeTag) {
+                        return typeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.IS_MISSING;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
index a528dae..3ae3bd9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
@@ -18,25 +18,17 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.DataOutput;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 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;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-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;
 
 public class IsNullDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
@@ -51,32 +43,16 @@ public class IsNullDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
             throws AlgebricksException {
         return new IScalarEvaluatorFactory() {
-
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
-                return new IScalarEvaluator() {
-
-                    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-                    private DataOutput out = resultStorage.getDataOutput();
-                    private IPointable argPtr = new VoidPointable();
-                    private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-                    private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
+                final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+                return new AbstractTypeCheckEvaluator(eval) {
 
                     @Override
-                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-                        eval.evaluate(tuple, argPtr);
-                        boolean isNull = argPtr.getByteArray()[argPtr
-                                .getStartOffset()] == ATypeTag.SERIALIZED_NULL_TYPE_TAG;
-                        ABoolean res = isNull ? ABoolean.TRUE : ABoolean.FALSE;
-                        try {
-                            resultStorage.reset();
-                            aObjSerDer.serialize(res, out);
-                            result.set(resultStorage);
-                        } catch (HyracksDataException e) {
-                            throw new AlgebricksException(e);
-                        }
+                    protected boolean isMatch(byte typeTag) {
+                        return typeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG;
                     }
                 };
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
new file mode 100644
index 0000000..be04c86
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.evaluators.functions;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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;
+
+public class IsUnknownDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new IsUnknownDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+                final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+                return new AbstractTypeCheckEvaluator(eval) {
+
+                    @Override
+                    protected boolean isMatch(byte typeTag) {
+                        return typeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG
+                                || typeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.IS_UNKOWN;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
index d42d96f..c681d78 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -26,7 +26,6 @@ import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializer
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -40,7 +39,6 @@ import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -77,9 +75,6 @@ public class LenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                     @SuppressWarnings("unchecked")
                     private final ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.AINT64);
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
@@ -88,16 +83,6 @@ public class LenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                         byte[] serList = inputVal.getByteArray();
                         int offset = inputVal.getStartOffset();
 
-                        if (serList[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                            try {
-                                nullSerde.serialize(ANull.NULL, out);
-                            } catch (HyracksDataException e) {
-                                throw new AlgebricksException(e);
-                            }
-                            result.set(resultStorage);
-                            return;
-                        }
-
                         if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
                             throw new AlgebricksException(AsterixBuiltinFunctions.LEN.getName()

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/535d86b5/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
index f05c2ed..fb209a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -23,7 +23,6 @@ import java.io.DataOutput;
 import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -77,9 +76,6 @@ public class NotDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
@@ -94,8 +90,6 @@ public class NotDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                                 boolean argRes = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
                                 ABoolean aResult = argRes ? (ABoolean.FALSE) : (ABoolean.TRUE);
                                 booleanSerde.serialize(aResult, out);
-                            } else if (bytes[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, out);
                             } else {
                                 throw new AlgebricksException(errorMessage);
                             }