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/04/23 02:24:34 UTC

[1/2] incubator-asterixdb git commit: Bytecode generation for null-handling in SQL++ functions.

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master eb2170ea1 -> dbd828587


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/CodeGenUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/CodeGenUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/CodeGenUtil.java
new file mode 100644
index 0000000..3de9fc4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/CodeGenUtil.java
@@ -0,0 +1,352 @@
+/*
+ * 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.staticcodegen;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassWriter;
+
+/**
+ * A utility class that generates byte code for scalar function descriptors.
+ */
+public class CodeGenUtil {
+
+    public final static String DEFAULT_SUFFIX_FOR_GENERATED_CLASS = "Gen";
+    private final static String OBJECT_CLASS_NAME = "java/lang/Object";
+    private final static String EVALUATOR_FACTORY = "EvaluatorFactory";
+    private final static String EVALUATOR = "Evaluator";
+    private final static String INNER = "Inner";
+    private final static String DESCRIPTOR = "Descriptor";
+    private final static String DOLLAR = "$";
+
+    /**
+     * The callback interface for a caller to determine what it needs to do for
+     * the generated class bytes.
+     */
+    public static interface ClassByteCodeAction {
+
+        /**
+         * Run a user-defined action for the generated class definition bytes.
+         *
+         * @param targetClassName,
+         *            the name for the generated class.
+         * @param classDefinitionBytes,
+         *            the class definition bytes.
+         * @throws IOException
+         */
+        public void runAction(String targetClassName, byte[] classDefinitionBytes) throws IOException;
+    };
+
+    /**
+     * Generates the byte code for a scalar function descriptor.
+     *
+     * @param packagePrefix,
+     *            the prefix of evaluators for code generation.
+     * @param originalFuncDescriptorClassName,
+     *            the original class name of the function descriptor.
+     * @param suffixForGeneratedClass,
+     *            the suffix for the generated class.
+     * @param action,
+     *            the customized action for the generated class definition bytes.
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    public static void generateScalarFunctionDescriptorBinary(String packagePrefix,
+            String originalFuncDescriptorClassName, String suffixForGeneratedClass, ClassLoader classLoader,
+            ClassByteCodeAction action) throws IOException, ClassNotFoundException {
+        originalFuncDescriptorClassName = toInternalClassName(originalFuncDescriptorClassName);
+        String targetFuncDescriptorClassName = toInternalClassName(
+                originalFuncDescriptorClassName + suffixForGeneratedClass);
+
+        // Adds the mapping of the old/new names of the function descriptor.
+        List<Pair<String, String>> nameMappings = new ArrayList<>();
+        nameMappings.add(Pair.of(originalFuncDescriptorClassName, targetFuncDescriptorClassName));
+        nameMappings.add(Pair.of(toJdkStandardName(originalFuncDescriptorClassName),
+                toJdkStandardName(targetFuncDescriptorClassName)));
+
+        // Gathers evaluator factory classes that are created in the function descriptor.
+        ClassReader reader = new ClassReader(getResourceStream(originalFuncDescriptorClassName, classLoader));
+        GatherEvaluatorFactoryCreationVisitor evalFactoryCreationVisitor = new GatherEvaluatorFactoryCreationVisitor(
+                toInternalClassName(packagePrefix));
+        reader.accept(evalFactoryCreationVisitor, 0);
+        Set<String> evaluatorFactoryClassNames = evalFactoryCreationVisitor.getCreatedEvaluatorFactoryClassNames();
+
+        // Generates inner classes other than evaluator factories.
+        generateNonEvalInnerClasses(reader, evaluatorFactoryClassNames, nameMappings, suffixForGeneratedClass,
+                classLoader, action);
+
+        // Generates evaluator factories that are created in the function descriptor.
+        int evalFactoryCounter = 0;
+        for (String evaluateFactoryClassName : evaluatorFactoryClassNames) {
+            generateEvaluatorFactoryClassBinary(packagePrefix, evaluateFactoryClassName, suffixForGeneratedClass,
+                    ++evalFactoryCounter, nameMappings, classLoader, action);
+        }
+
+        // Transforms the function descriptor class and outputs the generated class binary.
+        ClassWriter writer = new ClassWriter(reader, 0);
+        RenameClassVisitor renamingVisitor = new RenameClassVisitor(writer, nameMappings);
+        reader.accept(renamingVisitor, 0);
+        action.runAction(targetFuncDescriptorClassName, writer.toByteArray());
+    }
+
+    /**
+     * Apply mappings for a class name.
+     *
+     * @param nameMappings,
+     *            the mappings from existing class names to that of their generated counterparts.
+     * @param inputStr,
+     *            the name of a class.
+     * @return the name of the generated counterpart class.
+     */
+    static String applyMapping(List<Pair<String, String>> nameMappings, String inputStr) {
+        if (inputStr == null) {
+            return null;
+        }
+        String result = inputStr;
+
+        // Applies name mappings in the reverse order, i.e.,
+        // mapping recent added old/new name pairs first.
+        int index = nameMappings.size() - 1;
+        for (; index >= 0; --index) {
+            Pair<String, String> entry = nameMappings.get(index);
+            result = result.replace(entry.getLeft(), entry.getRight());
+        }
+        return result;
+    }
+
+    /**
+     * Generates the byte code for an evaluator factory class.
+     *
+     * @param packagePrefix,
+     *            the prefix of evaluators for code generation.
+     * @param originalEvaluatorFactoryClassName,
+     *            the original evaluator factory class name.
+     * @param suffixForGeneratedClass,
+     *            the suffix for the generated class.
+     * @param factoryIndex,
+     *            the counter for the generated class.
+     * @param nameMappings,
+     *            class names that needs to be rewritten in the generated byte code.
+     * @param classLoader,
+     *            a class loader that has the original evaluator factory class in its resource paths.
+     * @param action,
+     *            a user definition action for the generated byte code.
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    private static void generateEvaluatorFactoryClassBinary(String packagePrefix,
+            String originalEvaluatorFactoryClassName, String suffixForGeneratedClass, int factoryCounter,
+            List<Pair<String, String>> nameMappings, ClassLoader classLoader, ClassByteCodeAction action)
+            throws IOException, ClassNotFoundException {
+        originalEvaluatorFactoryClassName = toInternalClassName(originalEvaluatorFactoryClassName);
+        String targetEvaluatorFactoryClassName = getGeneratedClassName(originalEvaluatorFactoryClassName,
+                EVALUATOR_FACTORY + suffixForGeneratedClass, factoryCounter);
+
+        // Adds the old/new names of the evaluator factory into the mapping.
+        nameMappings.add(Pair.of(originalEvaluatorFactoryClassName, targetEvaluatorFactoryClassName));
+        nameMappings.add(Pair.of(toJdkStandardName(originalEvaluatorFactoryClassName),
+                toJdkStandardName(targetEvaluatorFactoryClassName)));
+
+        // Gathers the class names of the evaluators that are created in the evaluator factory.
+        ClassReader reader = new ClassReader(getResourceStream(originalEvaluatorFactoryClassName, classLoader));
+        GatherEvaluatorCreationVisitor evalCreationVisitor = new GatherEvaluatorCreationVisitor(
+                toInternalClassName(packagePrefix));
+        reader.accept(evalCreationVisitor, 0);
+        Set<String> evaluatorClassNames = evalCreationVisitor.getCreatedEvaluatorClassNames();
+
+        // Generates inner classes other than evaluators.
+        generateNonEvalInnerClasses(reader, evaluatorClassNames, nameMappings, suffixForGeneratedClass, classLoader,
+                action);
+
+        // Generates code for all evaluators.
+        int evalCounter = 0;
+        for (String evaluateClassName : evaluatorClassNames) {
+            generateEvaluatorClassBinary(evaluateClassName, suffixForGeneratedClass, ++evalCounter, nameMappings,
+                    classLoader, action);
+        }
+
+        // Transforms the evaluator factory class and outputs the generated class binary.
+        ClassWriter writer = new ClassWriter(reader, 0);
+        RenameClassVisitor renamingVisitor = new RenameClassVisitor(writer, nameMappings);
+        reader.accept(renamingVisitor, 0);
+        action.runAction(targetEvaluatorFactoryClassName, writer.toByteArray());
+    }
+
+    /**
+     * Generates the byte code for an evaluator class.
+     *
+     * @param originalEvaluatorClassName,
+     *            the name of the original evaluator class.
+     * @param suffixForGeneratedClass,
+     *            the suffix for the generated class.
+     * @param evalIndex,
+     *            the counter for the generated class.
+     * @param nameMappings,
+     *            class names that needs to be rewritten in the generated byte code.
+     * @param classLoader,
+     *            a class loader that has the original evaluator factory class in its resource paths.
+     * @param action,
+     *            a user definition action for the generated byte code.
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    private static void generateEvaluatorClassBinary(String originalEvaluatorClassName, String suffixForGeneratedClass,
+            int evalCounter, List<Pair<String, String>> nameMappings, ClassLoader classLoader,
+            ClassByteCodeAction action) throws IOException, ClassNotFoundException {
+        // Convert class names.
+        originalEvaluatorClassName = toInternalClassName(originalEvaluatorClassName);
+        if (originalEvaluatorClassName.equals(OBJECT_CLASS_NAME)) {
+            return;
+        }
+        String targetEvaluatorClassName = getGeneratedClassName(originalEvaluatorClassName,
+                EVALUATOR + suffixForGeneratedClass, evalCounter);
+
+        // Generates code for super classes except java.lang.Object.
+        Class<?> evaluatorClass = CodeGenUtil.class.getClassLoader()
+                .loadClass(toJdkStandardName(originalEvaluatorClassName));
+        generateEvaluatorClassBinary(evaluatorClass.getSuperclass().getName(), suffixForGeneratedClass, evalCounter,
+                nameMappings, classLoader, action);
+
+        // Adds name mapping.
+        nameMappings.add(Pair.of(originalEvaluatorClassName, targetEvaluatorClassName));
+        nameMappings.add(
+                Pair.of(toJdkStandardName(originalEvaluatorClassName), toJdkStandardName(targetEvaluatorClassName)));
+
+        // Injects null-handling byte code and output the class binary.
+        ClassReader reader = new ClassReader(getResourceStream(originalEvaluatorClassName, classLoader));
+        ClassWriter writer = new ClassWriter(reader, 0);
+        RenameClassVisitor renamingVisitor = new RenameClassVisitor(writer, nameMappings);
+        EvaluatorVisitor evaluateVisitor = new EvaluatorVisitor(renamingVisitor);
+        reader.accept(evaluateVisitor, 0);
+        action.runAction(targetEvaluatorClassName, writer.toByteArray());
+    }
+
+    /**
+     * Generates non-evaluator(-factory) inner classes defined in either a function descriptor
+     * or an evaluator factory.
+     *
+     * @param reader,
+     *            the reader of the outer class.
+     * @param evalClassNames,
+     *            the names of evaluator/evaluator-factory classes that shouldn't be generated in this
+     *            method.
+     * @param nameMappings,
+     *            class names that needs to be rewritten in the generated byte code.
+     * @param classLoader,
+     *            a class loader that has the original evaluator factory class in its resource paths.
+     * @param action,
+     *            a user definition action for the generated byte code.
+     * @throws IOException
+     */
+    private static void generateNonEvalInnerClasses(ClassReader reader, Set<String> evalClassNames,
+            List<Pair<String, String>> nameMappings, String suffixForGeneratedClass, ClassLoader classLoader,
+            ClassByteCodeAction action) throws IOException {
+        // Gathers inner classes of the function descriptor.
+        GatherInnerClassVisitor innerClassVisitor = new GatherInnerClassVisitor();
+        reader.accept(innerClassVisitor, 0);
+        Set<String> innerClassNames = innerClassVisitor.getInnerClassNames();
+        innerClassNames.removeAll(evalClassNames);
+
+        // Rewrites inner classes.
+        int counter = 0;
+        String suffix = INNER + suffixForGeneratedClass;
+        for (String innerClassName : innerClassNames) {
+            // adds name mapping.
+            String targetInnerClassName = getGeneratedClassName(innerClassName, suffix, ++counter);
+            nameMappings.add(Pair.of(innerClassName, targetInnerClassName));
+            nameMappings.add(Pair.of(toJdkStandardName(innerClassName), toJdkStandardName(targetInnerClassName)));
+
+            // Renaming appearances of original class names.
+            ClassReader innerClassReader = new ClassReader(getResourceStream(innerClassName, classLoader));
+            ClassWriter writer = new ClassWriter(innerClassReader, 0);
+            RenameClassVisitor renamingVisitor = new RenameClassVisitor(writer, nameMappings);
+            innerClassReader.accept(renamingVisitor, 0);
+            action.runAction(targetInnerClassName, writer.toByteArray());
+        }
+    }
+
+    /**
+     * Converts a JDK class name to the class naming format of ASM.
+     *
+     * @param name,
+     *            a class name following the JDK convention.
+     * @return a "/"-separated class name assumed by ASM.
+     */
+    private static String toInternalClassName(String name) {
+        return name.replace(".", "/");
+    }
+
+    /**
+     * Converts an ASM class name to the JDK class naming format.
+     *
+     * @param name,
+     *            a class name following the ASM convention.
+     * @return a "."-separated class name for JDK.
+     */
+    private static String toJdkStandardName(String name) {
+        return name.replace("/", ".");
+    }
+
+    /**
+     * Gets the name of a generated class.
+     *
+     * @param originalClassName,
+     *            the original class, i.e., the source of the generated class.
+     * @param suffix,
+     *            the suffix for the generated class.
+     * @param evalCounter,
+     *            a counter that appearing at the end of the name of the generated class.
+     * @return the name of the generated class.
+     */
+    private static String getGeneratedClassName(String originalClassName, String suffix, int counter) {
+        StringBuilder sb = new StringBuilder();
+        int end = originalClassName.indexOf(DESCRIPTOR);
+        if (end < 0) {
+            end = originalClassName.indexOf(DOLLAR);
+        }
+        if (end < 0) {
+            end = originalClassName.length() - 1;
+        }
+
+        sb.append(originalClassName.substring(0, end));
+        sb.append(suffix);
+        sb.append(counter);
+        return sb.toString();
+    }
+
+    /**
+     * Gets the input stream from a class file.
+     *
+     * @param className,
+     *            the name of a class.
+     * @param classLoader,
+     *            the corresponding class loader.
+     * @return the input stream.
+     */
+    private static InputStream getResourceStream(String className, ClassLoader classLoader) {
+        return classLoader.getResourceAsStream(className.replace('.', '/') + ".class");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorVisitor.java
new file mode 100644
index 0000000..6d554d7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorVisitor.java
@@ -0,0 +1,125 @@
+/*
+ * 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.staticcodegen;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.tree.AbstractInsnNode;
+import org.objectweb.asm.tree.FieldInsnNode;
+import org.objectweb.asm.tree.IincInsnNode;
+import org.objectweb.asm.tree.InsnNode;
+import org.objectweb.asm.tree.IntInsnNode;
+
+/**
+ * This visitor adds null-handling byte code into an evaluator class.
+ */
+public class EvaluatorVisitor extends ClassVisitor {
+    private final static String EVALUATE_DESC = "(Lorg/apache/hyracks/dataflow/common/data/accessors/IFrameTupleReference;Lorg/apache/hyracks/data/std/api/IPointable;)V";
+    private final static String EVALUATE = "evaluate";
+    private final static MethodIdentifier METHOD_IDENTIFIER = new MethodIdentifier(EVALUATE, EVALUATE_DESC, null);
+    private final static String TYPECHECK_CLASS = "org/apache/asterix/runtime/evaluators/staticcodegen/TypeCheckUtil";
+    private final static String IS_NULL = "isNull";
+    private final static String TYPECHECK_METHOD_DESC = "(Lorg/apache/hyracks/data/std/api/IPointable;Lorg/apache/hyracks/data/std/api/IPointable;)Z";
+
+    public EvaluatorVisitor(ClassVisitor downStreamVisitor) {
+        super(Opcodes.ASM5, downStreamVisitor);
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions) {
+        MethodVisitor mv = cv.visitMethod(access, name, desc, signature, exceptions);
+        if (!METHOD_IDENTIFIER.equals(new MethodIdentifier(name, desc, signature))) {
+            return mv;
+        }
+        if (mv != null) {
+            return new MethodVisitor(Opcodes.ASM5, mv) {
+                private FieldInsnNode fieldAccessNode = null;
+                private List<AbstractInsnNode> instructionsAfterFieldAccess = new ArrayList<>();
+
+                @Override
+                public void visitFieldInsn(int opcode, String owner, String name, String desc) {
+                    mv.visitFieldInsn(opcode, owner, name, desc);
+                    fieldAccessNode = new FieldInsnNode(opcode, owner, name, desc);
+                    instructionsAfterFieldAccess.clear();
+                }
+
+                @Override
+                public void visitIincInsn(int var, int increment) {
+                    if (fieldAccessNode != null) {
+                        instructionsAfterFieldAccess.add(new IincInsnNode(var, increment));
+                    }
+                    super.visitIincInsn(var, increment);
+                }
+
+                @Override
+                public void visitInsn(int opcode) {
+                    if (fieldAccessNode != null) {
+                        instructionsAfterFieldAccess.add(new InsnNode(opcode));
+                    }
+                    super.visitInsn(opcode);
+                }
+
+                @Override
+                public void visitIntInsn(int opcode, int operand) {
+                    if (fieldAccessNode != null) {
+                        instructionsAfterFieldAccess.add(new IntInsnNode(opcode, operand));
+                    }
+                    super.visitIntInsn(opcode, operand);
+                }
+
+                @Override
+                public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf) {
+                    mv.visitMethodInsn(opcode, owner, name, desc, itf);
+                    if (fieldAccessNode == null
+                            || !METHOD_IDENTIFIER.equals(new MethodIdentifier(name, desc, signature))) {
+                        return;
+                    }
+                    // Loads "this".
+                    mv.visitVarInsn(Opcodes.ALOAD, 0);
+                    // Replays the field access instruction.
+                    fieldAccessNode.accept(mv);
+
+                    // Replays other instruction between the field access and the evaluator call.
+                    for (AbstractInsnNode instruction : instructionsAfterFieldAccess) {
+                        instruction.accept(mv);
+                    }
+
+                    // Loads the result IPointable.
+                    mv.visitVarInsn(Opcodes.ALOAD, 2);
+
+                    // Invokes the null check method.
+                    mv.visitMethodInsn(Opcodes.INVOKESTATIC, TYPECHECK_CLASS, IS_NULL, TYPECHECK_METHOD_DESC, false);
+                    Label notNull = new Label();
+                    // Adds the if branch.
+                    mv.visitJumpInsn(Opcodes.IFEQ, notNull);
+                    mv.visitInsn(Opcodes.RETURN);
+                    mv.visitLabel(notNull);
+                    mv.visitFrame(Opcodes.F_SAME, 0, null, 0, null);
+                }
+            };
+        }
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorCreationVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorCreationVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorCreationVisitor.java
new file mode 100644
index 0000000..416ce63
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorCreationVisitor.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.staticcodegen;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+/**
+ * This visitor gathers all created evaluators in an evaluator factory.
+ */
+public class GatherEvaluatorCreationVisitor extends ClassVisitor {
+
+    private static String METHOD_NAME = "createScalarEvaluator";
+    private Set<String> createdEvaluatorClassNames = new HashSet<>();
+    private String ownerPrefix;
+
+    public GatherEvaluatorCreationVisitor(String ownerPrefix) {
+        super(Opcodes.ASM5);
+        this.ownerPrefix = ownerPrefix;
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions) {
+        if (!name.equals(METHOD_NAME)) {
+            return null;
+        }
+        return new MethodVisitor(Opcodes.ASM5, null) {
+
+            @Override
+            public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf) {
+                if (opcode != Opcodes.INVOKESPECIAL) {
+                    return;
+                }
+                if (owner.startsWith(ownerPrefix)) {
+                    createdEvaluatorClassNames.add(owner);
+                }
+            }
+        };
+
+    }
+
+    public Set<String> getCreatedEvaluatorClassNames() {
+        return createdEvaluatorClassNames;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorFactoryCreationVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorFactoryCreationVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorFactoryCreationVisitor.java
new file mode 100644
index 0000000..19234e2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherEvaluatorFactoryCreationVisitor.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.staticcodegen;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+/**
+ * This visitor gathers all created evaluator factories in a scalar function descriptor.
+ */
+public class GatherEvaluatorFactoryCreationVisitor extends ClassVisitor {
+
+    private static String METHOD_NAME = "createEvaluatorFactory";
+    private final Set<String> createdEvaluatorFactoryClassNames = new HashSet<>();
+    private String ownerPrefix;
+
+    public GatherEvaluatorFactoryCreationVisitor(String ownerPrefix) {
+        super(Opcodes.ASM5);
+        this.ownerPrefix = ownerPrefix;
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions) {
+        if (!name.equals(METHOD_NAME)) {
+            return null;
+        }
+        return new MethodVisitor(Opcodes.ASM5, null) {
+
+            @Override
+            public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf) {
+                if (opcode != Opcodes.INVOKESPECIAL) {
+                    return;
+                }
+                if (owner.startsWith(ownerPrefix)) {
+                    createdEvaluatorFactoryClassNames.add(owner);
+                }
+            }
+        };
+    }
+
+    public Set<String> getCreatedEvaluatorFactoryClassNames() {
+        return createdEvaluatorFactoryClassNames;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
new file mode 100644
index 0000000..2293734
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.staticcodegen;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.Opcodes;
+
+/**
+ * This class gathers all inner classes defined in a class.
+ */
+public class GatherInnerClassVisitor extends ClassVisitor {
+
+    private final Set<String> innerClassNames = new HashSet<>();
+    private String className = null;
+
+    public GatherInnerClassVisitor() {
+        super(Opcodes.ASM5);
+    }
+
+    @Override
+    public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
+        className = name;
+    }
+
+    @Override
+    public void visitInnerClass(String name, String outerName, String innerName, int access) {
+        if (className == null || !name.equals(className)) {
+            innerClassNames.add(name);
+        }
+    }
+
+    public Set<String> getInnerClassNames() {
+        return innerClassNames;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/MethodIdentifier.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/MethodIdentifier.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/MethodIdentifier.java
new file mode 100644
index 0000000..c37b93e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/MethodIdentifier.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.runtime.evaluators.staticcodegen;
+
+/**
+ * The data structure that can uniquely identify a method.
+ */
+public class MethodIdentifier {
+
+    private final String name;
+    private final String desc;
+    private final String signature;
+
+    public MethodIdentifier(String name, String desc, String signature) {
+        this.name = name == null ? "" : name;
+        this.desc = desc == null ? "" : desc;
+        this.signature = signature == null ? "" : signature;
+    }
+
+    @Override
+    public int hashCode() {
+        return name.hashCode() * desc.hashCode() * signature.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof MethodIdentifier)) {
+            return false;
+        }
+        MethodIdentifier methodIdentifier = (MethodIdentifier) o;
+        return name.equals(methodIdentifier.name) && desc.equals(methodIdentifier.desc)
+                && signature.equals(methodIdentifier.signature);
+    }
+
+    @Override
+    public String toString() {
+        return name + ":" + desc + ":" + signature;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/RenameClassVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/RenameClassVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/RenameClassVisitor.java
new file mode 100644
index 0000000..1eb7357
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/RenameClassVisitor.java
@@ -0,0 +1,112 @@
+/*
+ * 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.staticcodegen;
+
+import java.util.List;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+/**
+ * This visitor replaces all the appearances of original class names with
+ * new (generated) class names, according to an input name mapping.
+ */
+public class RenameClassVisitor extends ClassVisitor {
+
+    private final List<Pair<String, String>> nameMapping;
+
+    public RenameClassVisitor(ClassVisitor downStreamVisitor, List<Pair<String, String>> nameMapping) {
+        super(Opcodes.ASM5, downStreamVisitor);
+        this.nameMapping = nameMapping;
+    }
+
+    @Override
+    public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
+        super.visit(version, access, applyMapping(name), signature, applyMapping(superName), interfaces);
+    }
+
+    @Override
+    public void visitOuterClass(String owner, String name, String desc) {
+        // Skips outer class descriptions.
+    }
+
+    @Override
+    public void visitInnerClass(String name, String outerName, String innerName, int access) {
+        // Skips inner class descriptions.
+    }
+
+    @Override
+    public FieldVisitor visitField(int access, String name, String desc, String signature, Object value) {
+        return cv.visitField(access, name, applyMapping(desc), applyMapping(signature), value);
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions) {
+        MethodVisitor mv = cv.visitMethod(access, name, applyMapping(desc), applyMapping(signature), exceptions);
+        if (mv != null) {
+            return new MethodVisitor(Opcodes.ASM5, mv) {
+
+                @Override
+                public void visitFieldInsn(int opcode, String owner, String name, String desc) {
+                    mv.visitFieldInsn(opcode, applyMapping(owner), applyMapping(name), applyMapping(desc));
+                }
+
+                @Override
+                public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf) {
+                    mv.visitMethodInsn(opcode, applyMapping(owner), name, applyMapping(desc), itf);
+                }
+
+                @Override
+                public void visitLocalVariable(String name, String desc, String signature, Label start, Label end,
+                        int index) {
+                    mv.visitLocalVariable(name, applyMapping(desc), applyMapping(signature), start, end, index);
+                }
+
+                @Override
+                public void visitFrame(int type, int nLocal, Object[] local, int nStack, Object[] stack) {
+                    if (local != null) {
+                        for (int index = 0; index < local.length; ++index) {
+                            if (local[index] instanceof String) {
+                                local[index] = applyMapping((String) local[index]);
+                            }
+                        }
+                    }
+                    mv.visitFrame(type, nLocal, local, nStack, stack);
+                }
+
+                @Override
+                public void visitTypeInsn(int opcode, String type) {
+                    mv.visitTypeInsn(opcode, applyMapping(type));
+                }
+
+            };
+        }
+        return null;
+    }
+
+    private String applyMapping(String inputStr) {
+        return CodeGenUtil.applyMapping(nameMapping, inputStr);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/TypeCheckUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/TypeCheckUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/TypeCheckUtil.java
new file mode 100644
index 0000000..0b02f39
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/TypeCheckUtil.java
@@ -0,0 +1,41 @@
+/*
+ * 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.staticcodegen;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * The null-handling code to be injected into the evaluator(...) method of scalar evaluators.
+ */
+public class TypeCheckUtil {
+
+    public static byte[] NULL_BYTES = new byte[] { ATypeTag.SERIALIZED_NULL_TYPE_TAG };
+
+    public static boolean isNull(IPointable arg, IPointable resultPointable) {
+        byte[] data = arg.getByteArray();
+        int start = arg.getStartOffset();
+        if (data[start] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+            resultPointable.set(NULL_BYTES, 0, 1);
+            return true;
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index 054f3e4..3e187a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -65,197 +65,16 @@ import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.aggregates.collections.ListifyAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarCountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarMaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarMinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.scalar.ScalarSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableCountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSqlSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlCountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.IntermediateAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.IntermediateSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalMaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalMinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalSqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalSqlMaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalSqlMinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalSqlSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.LocalSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.MinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SqlAvgAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SqlMinAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
-import org.apache.asterix.runtime.aggregates.stream.NonEmptyStreamAggregateDescriptor;
-import org.apache.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalDayAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalHourAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndDateAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndDatetimeAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndTimeAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartDateAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartDatetimeAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartTimeAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalMillisecondAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalMinuteAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalMonthAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalSecondAccessor;
-import org.apache.asterix.runtime.evaluators.accessors.TemporalYearAccessor;
 import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
 import org.apache.asterix.runtime.evaluators.common.FunctionManagerImpl;
-import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AFloatConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AInt16ConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AInt32ConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AInt64ConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AInt8ConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AIntervalConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateTimeConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromTimeConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ALineConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ANullConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.APoint3DConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.APointConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.APolygonConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.OpenRecordConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastListDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CountHashedGramTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CountHashedWordTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateCircleDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateLineDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateMBRDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreatePointDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreatePolygonDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateQueryUIDDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateRectangleDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.CreateUUIDDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.EditDistanceCheckDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.EditDistanceContainsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.EditDistanceDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.EditDistanceListIsFilterable;
-import org.apache.asterix.runtime.evaluators.functions.EditDistanceStringIsFilterable;
-import org.apache.asterix.runtime.evaluators.functions.EmbedTypeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FlowRecordDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.FuzzyEqDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.GramTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.HashedGramTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.HashedWordTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.InjectFailureDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.IsNullDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.IsSystemNullDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.LenDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NotNullDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericAddDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericSubDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.OrderedListConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PrefixLenJaccardDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.RegExpDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardCheckDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixCheckDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedCheckDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringConcatDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringEqualDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringJoinDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringLengthDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringLikeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringLowerCaseDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringMatchesDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringMatchesWithFlagDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithFlagsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringUpperCaseDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.Substring2Descriptor;
-import org.apache.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SubstringDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.WordTokensDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.BinaryConcatDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.BinaryLengthDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.FindBinaryDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.FindBinaryFromDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.ParseBinaryDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.PrintBinaryDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.SubBinaryFromDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.binary.SubBinaryFromToDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByIndexDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByIndexEvalFactory;
 import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByNameDescriptor;
@@ -266,57 +85,6 @@ import org.apache.asterix.runtime.evaluators.functions.records.GetRecordFieldsDe
 import org.apache.asterix.runtime.evaluators.functions.records.RecordAddFieldsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordMergeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveFieldsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustDateTimeForTimeZoneDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustTimeForTimeZoneDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDuartionFromDateDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDurationFromDateTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromDatetimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromUnixTimeInDaysDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromDateAndTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInMsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInSecsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DayOfWeekDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DayTimeDurationComparatorDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DurationEqualDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromIntervalDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromMillisecondsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromMonthsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.GetDayTimeDurationDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.GetOverlappingIntervalDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.GetYearMonthDurationDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalAfterDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalBeforeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalBinDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalCoveredByDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalCoversDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalEndedByDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalEndsDecriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMeetsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMetByDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlappedByDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapBinsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.ParseTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.PrintTimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
-import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
-import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
-import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
-import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -384,7 +152,7 @@ public class NonTaggedDataFormat implements IDataFormat {
     }
 
     @Override
-    public void registerRuntimeFunctions() throws AlgebricksException {
+    public void registerRuntimeFunctions(List<IFunctionDescriptorFactory> funcDescriptors) throws AlgebricksException {
 
         if (registered) {
             return;
@@ -395,303 +163,8 @@ public class NonTaggedDataFormat implements IDataFormat {
             return;
         }
 
-        List<IFunctionDescriptorFactory> temp = new ArrayList<IFunctionDescriptorFactory>();
-
-        // format-independent
-        temp.add(TidRunningAggregateDescriptor.FACTORY);
-
-        // format-dependent
-        temp.add(AndDescriptor.FACTORY);
-        temp.add(OrDescriptor.FACTORY);
-        temp.add(StringLikeDescriptor.FACTORY);
-        temp.add(ScanCollectionDescriptor.FACTORY);
-        temp.add(AnyCollectionMemberDescriptor.FACTORY);
-        temp.add(ClosedRecordConstructorDescriptor.FACTORY);
-        temp.add(FieldAccessByIndexDescriptor.FACTORY);
-        temp.add(FieldAccessByNameDescriptor.FACTORY);
-        temp.add(FieldAccessNestedDescriptor.FACTORY);
-        temp.add(GetRecordFieldsDescriptor.FACTORY);
-        temp.add(GetRecordFieldValueDescriptor.FACTORY);
-        temp.add(FieldAccessByNameDescriptor.FACTORY);
-        temp.add(GetItemDescriptor.FACTORY);
-        temp.add(NumericUnaryMinusDescriptor.FACTORY);
-        temp.add(OpenRecordConstructorDescriptor.FACTORY);
-        temp.add(OrderedListConstructorDescriptor.FACTORY);
-        temp.add(UnorderedListConstructorDescriptor.FACTORY);
-        temp.add(EmbedTypeDescriptor.FACTORY);
-
-        temp.add(NumericAddDescriptor.FACTORY);
-        temp.add(NumericDivideDescriptor.FACTORY);
-        temp.add(NumericMultiplyDescriptor.FACTORY);
-        temp.add(NumericSubDescriptor.FACTORY);
-        temp.add(NumericModuloDescriptor.FACTORY);
-        temp.add(NumericCaretDescriptor.FACTORY);
-        temp.add(IsNullDescriptor.FACTORY);
-        temp.add(IsSystemNullDescriptor.FACTORY);
-        temp.add(NotDescriptor.FACTORY);
-        temp.add(LenDescriptor.FACTORY);
-        temp.add(EmptyStreamAggregateDescriptor.FACTORY);
-        temp.add(NonEmptyStreamAggregateDescriptor.FACTORY);
-        temp.add(RangeDescriptor.FACTORY);
-
-        temp.add(NumericAbsDescriptor.FACTORY);
-        temp.add(NumericCeilingDescriptor.FACTORY);
-        temp.add(NumericFloorDescriptor.FACTORY);
-        temp.add(NumericRoundDescriptor.FACTORY);
-        temp.add(NumericRoundHalfToEvenDescriptor.FACTORY);
-        temp.add(NumericRoundHalfToEven2Descriptor.FACTORY);
-
-        // Binary functions
-        temp.add(BinaryLengthDescriptor.FACTORY);
-        temp.add(ParseBinaryDescriptor.FACTORY);
-        temp.add(PrintBinaryDescriptor.FACTORY);
-        temp.add(BinaryConcatDescriptor.FACTORY);
-        temp.add(SubBinaryFromDescriptor.FACTORY);
-        temp.add(SubBinaryFromToDescriptor.FACTORY);
-        temp.add(FindBinaryDescriptor.FACTORY);
-        temp.add(FindBinaryFromDescriptor.FACTORY);
-
-        // String functions
-        temp.add(StringContainsDescriptor.FACTORY);
-        temp.add(StringEndsWithDescriptor.FACTORY);
-        temp.add(StringStartsWithDescriptor.FACTORY);
-        temp.add(SubstringDescriptor.FACTORY);
-        temp.add(StringEqualDescriptor.FACTORY);
-        temp.add(StringMatchesDescriptor.FACTORY);
-        temp.add(StringLowerCaseDescriptor.FACTORY);
-        temp.add(StringUpperCaseDescriptor.FACTORY);
-        temp.add(StringMatchesWithFlagDescriptor.FACTORY);
-        temp.add(StringReplaceDescriptor.FACTORY);
-        temp.add(StringReplaceWithFlagsDescriptor.FACTORY);
-        temp.add(StringLengthDescriptor.FACTORY);
-        temp.add(Substring2Descriptor.FACTORY);
-        temp.add(SubstringBeforeDescriptor.FACTORY);
-        temp.add(SubstringAfterDescriptor.FACTORY);
-        temp.add(StringToCodePointDescriptor.FACTORY);
-        temp.add(CodePointToStringDescriptor.FACTORY);
-        temp.add(StringConcatDescriptor.FACTORY);
-        temp.add(StringJoinDescriptor.FACTORY);
-
-        // aggregates
-        temp.add(ListifyAggregateDescriptor.FACTORY);
-        temp.add(CountAggregateDescriptor.FACTORY);
-        temp.add(AvgAggregateDescriptor.FACTORY);
-        temp.add(LocalAvgAggregateDescriptor.FACTORY);
-        temp.add(IntermediateAvgAggregateDescriptor.FACTORY);
-        temp.add(GlobalAvgAggregateDescriptor.FACTORY);
-        temp.add(SumAggregateDescriptor.FACTORY);
-        temp.add(LocalSumAggregateDescriptor.FACTORY);
-        temp.add(MaxAggregateDescriptor.FACTORY);
-        temp.add(LocalMaxAggregateDescriptor.FACTORY);
-        temp.add(MinAggregateDescriptor.FACTORY);
-        temp.add(LocalMinAggregateDescriptor.FACTORY);
-
-        // serializable aggregates
-        temp.add(SerializableCountAggregateDescriptor.FACTORY);
-        temp.add(SerializableAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableLocalAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableIntermediateAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableGlobalAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableSumAggregateDescriptor.FACTORY);
-        temp.add(SerializableLocalSumAggregateDescriptor.FACTORY);
-
-        // scalar aggregates
-        temp.add(ScalarCountAggregateDescriptor.FACTORY);
-        temp.add(ScalarAvgAggregateDescriptor.FACTORY);
-        temp.add(ScalarSumAggregateDescriptor.FACTORY);
-        temp.add(ScalarMaxAggregateDescriptor.FACTORY);
-        temp.add(ScalarMinAggregateDescriptor.FACTORY);
-
-        // SQL aggregates
-        temp.add(SqlCountAggregateDescriptor.FACTORY);
-        temp.add(SqlAvgAggregateDescriptor.FACTORY);
-        temp.add(LocalSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(IntermediateSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(GlobalSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(SqlSumAggregateDescriptor.FACTORY);
-        temp.add(LocalSqlSumAggregateDescriptor.FACTORY);
-        temp.add(SqlMaxAggregateDescriptor.FACTORY);
-        temp.add(LocalSqlMaxAggregateDescriptor.FACTORY);
-        temp.add(SqlMinAggregateDescriptor.FACTORY);
-        temp.add(LocalSqlMinAggregateDescriptor.FACTORY);
-
-        // SQL serializable aggregates
-        temp.add(SerializableSqlCountAggregateDescriptor.FACTORY);
-        temp.add(SerializableSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableLocalSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableIntermediateSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableGlobalSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(SerializableSqlSumAggregateDescriptor.FACTORY);
-        temp.add(SerializableLocalSqlSumAggregateDescriptor.FACTORY);
-
-        // SQL scalar aggregates
-        temp.add(ScalarSqlCountAggregateDescriptor.FACTORY);
-        temp.add(ScalarSqlAvgAggregateDescriptor.FACTORY);
-        temp.add(ScalarSqlSumAggregateDescriptor.FACTORY);
-        temp.add(ScalarSqlMaxAggregateDescriptor.FACTORY);
-        temp.add(ScalarSqlMinAggregateDescriptor.FACTORY);
-
-        // new functions - constructors
-        temp.add(ABooleanConstructorDescriptor.FACTORY);
-        temp.add(ANullConstructorDescriptor.FACTORY);
-        temp.add(ABinaryHexStringConstructorDescriptor.FACTORY);
-        temp.add(ABinaryBase64StringConstructorDescriptor.FACTORY);
-        temp.add(AStringConstructorDescriptor.FACTORY);
-        temp.add(AInt8ConstructorDescriptor.FACTORY);
-        temp.add(AInt16ConstructorDescriptor.FACTORY);
-        temp.add(AInt32ConstructorDescriptor.FACTORY);
-        temp.add(AInt64ConstructorDescriptor.FACTORY);
-        temp.add(AFloatConstructorDescriptor.FACTORY);
-        temp.add(ADoubleConstructorDescriptor.FACTORY);
-        temp.add(APointConstructorDescriptor.FACTORY);
-        temp.add(APoint3DConstructorDescriptor.FACTORY);
-        temp.add(ALineConstructorDescriptor.FACTORY);
-        temp.add(APolygonConstructorDescriptor.FACTORY);
-        temp.add(ACircleConstructorDescriptor.FACTORY);
-        temp.add(ARectangleConstructorDescriptor.FACTORY);
-        temp.add(ATimeConstructorDescriptor.FACTORY);
-        temp.add(ADateConstructorDescriptor.FACTORY);
-        temp.add(ADateTimeConstructorDescriptor.FACTORY);
-        temp.add(ADurationConstructorDescriptor.FACTORY);
-        temp.add(AYearMonthDurationConstructorDescriptor.FACTORY);
-        temp.add(ADayTimeDurationConstructorDescriptor.FACTORY);
-        temp.add(AUUIDFromStringConstructorDescriptor.FACTORY);
-
-        temp.add(DeepEqualityDescriptor.FACTORY);
-
-        temp.add(CreateUUIDDescriptor.FACTORY);
-        temp.add(CreateQueryUIDDescriptor.FACTORY);
-        // Spatial
-        temp.add(CreatePointDescriptor.FACTORY);
-        temp.add(CreateLineDescriptor.FACTORY);
-        temp.add(CreatePolygonDescriptor.FACTORY);
-        temp.add(CreateCircleDescriptor.FACTORY);
-        temp.add(CreateRectangleDescriptor.FACTORY);
-        temp.add(SpatialAreaDescriptor.FACTORY);
-        temp.add(SpatialDistanceDescriptor.FACTORY);
-        temp.add(SpatialIntersectDescriptor.FACTORY);
-        temp.add(CreateMBRDescriptor.FACTORY);
-        temp.add(SpatialCellDescriptor.FACTORY);
-        temp.add(PointXCoordinateAccessor.FACTORY);
-        temp.add(PointYCoordinateAccessor.FACTORY);
-        temp.add(CircleRadiusAccessor.FACTORY);
-        temp.add(CircleCenterAccessor.FACTORY);
-        temp.add(LineRectanglePolygonAccessor.FACTORY);
-
-        // fuzzyjoin function
-        temp.add(FuzzyEqDescriptor.FACTORY);
-        temp.add(SubsetCollectionDescriptor.FACTORY);
-        temp.add(PrefixLenJaccardDescriptor.FACTORY);
-
-        temp.add(WordTokensDescriptor.FACTORY);
-        temp.add(HashedWordTokensDescriptor.FACTORY);
-        temp.add(CountHashedWordTokensDescriptor.FACTORY);
-
-        temp.add(GramTokensDescriptor.FACTORY);
-        temp.add(HashedGramTokensDescriptor.FACTORY);
-        temp.add(CountHashedGramTokensDescriptor.FACTORY);
-
-        temp.add(EditDistanceDescriptor.FACTORY);
-        temp.add(EditDistanceCheckDescriptor.FACTORY);
-        temp.add(EditDistanceStringIsFilterable.FACTORY);
-        temp.add(EditDistanceListIsFilterable.FACTORY);
-        temp.add(EditDistanceContainsDescriptor.FACTORY);
-
-        temp.add(SimilarityJaccardDescriptor.FACTORY);
-        temp.add(SimilarityJaccardCheckDescriptor.FACTORY);
-        temp.add(SimilarityJaccardSortedDescriptor.FACTORY);
-        temp.add(SimilarityJaccardSortedCheckDescriptor.FACTORY);
-        temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
-        temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
-
-        //Record functions
-        temp.add(RecordMergeDescriptor.FACTORY);
-        temp.add(RecordAddFieldsDescriptor.FACTORY);
-        temp.add(RecordRemoveFieldsDescriptor.FACTORY);
-        temp.add(SwitchCaseDescriptor.FACTORY);
-        temp.add(RegExpDescriptor.FACTORY);
-        temp.add(InjectFailureDescriptor.FACTORY);
-        temp.add(CastListDescriptor.FACTORY);
-        temp.add(CastRecordDescriptor.FACTORY);
-        temp.add(FlowRecordDescriptor.FACTORY);
-        temp.add(NotNullDescriptor.FACTORY);
-
-        // Spatial and temporal type accessors
-        temp.add(TemporalYearAccessor.FACTORY);
-        temp.add(TemporalMonthAccessor.FACTORY);
-        temp.add(TemporalDayAccessor.FACTORY);
-        temp.add(TemporalHourAccessor.FACTORY);
-        temp.add(TemporalMinuteAccessor.FACTORY);
-        temp.add(TemporalSecondAccessor.FACTORY);
-        temp.add(TemporalMillisecondAccessor.FACTORY);
-        temp.add(TemporalIntervalStartAccessor.FACTORY);
-        temp.add(TemporalIntervalEndAccessor.FACTORY);
-        temp.add(TemporalIntervalStartDateAccessor.FACTORY);
-        temp.add(TemporalIntervalEndDateAccessor.FACTORY);
-        temp.add(TemporalIntervalStartTimeAccessor.FACTORY);
-        temp.add(TemporalIntervalEndTimeAccessor.FACTORY);
-        temp.add(TemporalIntervalStartDatetimeAccessor.FACTORY);
-        temp.add(TemporalIntervalEndDatetimeAccessor.FACTORY);
-
-        // Temporal functions
-        temp.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
-        temp.add(DateFromDatetimeDescriptor.FACTORY);
-        temp.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
-        temp.add(TimeFromDatetimeDescriptor.FACTORY);
-        temp.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
-        temp.add(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
-        temp.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
-        temp.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
-        temp.add(CalendarDuartionFromDateDescriptor.FACTORY);
-        temp.add(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
-        temp.add(AdjustTimeForTimeZoneDescriptor.FACTORY);
-        temp.add(IntervalBeforeDescriptor.FACTORY);
-        temp.add(IntervalAfterDescriptor.FACTORY);
-        temp.add(IntervalMeetsDescriptor.FACTORY);
-        temp.add(IntervalMetByDescriptor.FACTORY);
-        temp.add(IntervalOverlapsDescriptor.FACTORY);
-        temp.add(IntervalOverlappedByDescriptor.FACTORY);
-        temp.add(OverlapDescriptor.FACTORY);
-        temp.add(IntervalStartsDescriptor.FACTORY);
-        temp.add(IntervalStartedByDescriptor.FACTORY);
-        temp.add(IntervalCoversDescriptor.FACTORY);
-        temp.add(IntervalCoveredByDescriptor.FACTORY);
-        temp.add(IntervalEndsDecriptor.FACTORY);
-        temp.add(IntervalEndedByDescriptor.FACTORY);
-        temp.add(CurrentDateDescriptor.FACTORY);
-        temp.add(CurrentTimeDescriptor.FACTORY);
-        temp.add(CurrentDateTimeDescriptor.FACTORY);
-        temp.add(DurationFromMillisecondsDescriptor.FACTORY);
-        temp.add(DurationFromMonthsDescriptor.FACTORY);
-        temp.add(YearMonthDurationComparatorDecriptor.GREATER_THAN_FACTORY);
-        temp.add(YearMonthDurationComparatorDecriptor.LESS_THAN_FACTORY);
-        temp.add(DayTimeDurationComparatorDescriptor.GREATER_THAN_FACTORY);
-        temp.add(DayTimeDurationComparatorDescriptor.LESS_THAN_FACTORY);
-        temp.add(MonthsFromYearMonthDurationDescriptor.FACTORY);
-        temp.add(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
-        temp.add(DurationEqualDescriptor.FACTORY);
-        temp.add(GetYearMonthDurationDescriptor.FACTORY);
-        temp.add(GetDayTimeDurationDescriptor.FACTORY);
-        temp.add(IntervalBinDescriptor.FACTORY);
-        temp.add(OverlapBinsDescriptor.FACTORY);
-        temp.add(DayOfWeekDescriptor.FACTORY);
-        temp.add(ParseDateDescriptor.FACTORY);
-        temp.add(ParseTimeDescriptor.FACTORY);
-        temp.add(ParseDateTimeDescriptor.FACTORY);
-        temp.add(PrintDateDescriptor.FACTORY);
-        temp.add(PrintTimeDescriptor.FACTORY);
-        temp.add(PrintDateTimeDescriptor.FACTORY);
-        temp.add(GetOverlappingIntervalDescriptor.FACTORY);
-        temp.add(DurationFromIntervalDescriptor.FACTORY);
-
-        // Interval constructor
-        temp.add(AIntervalConstructorDescriptor.FACTORY);
-        temp.add(AIntervalStartFromDateConstructorDescriptor.FACTORY);
-        temp.add(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
-        temp.add(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
-
         IFunctionManager mgr = new FunctionManagerImpl();
-        for (IFunctionDescriptorFactory fdFactory : temp) {
+        for (IFunctionDescriptorFactory fdFactory : funcDescriptors) {
             mgr.registerFunction(fdFactory);
         }
         FunctionManagerHolder.setFunctionManager(mgr);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index dd7a8b3..4073247 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -248,6 +248,19 @@
                                 </pluginExecution>
                                 <pluginExecution>
                                     <pluginExecutionFilter>
+                                        <groupId>org.apache.asterix</groupId>
+                                        <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+                                        <versionRange>[0.8.7-SNAPSHOT,)</versionRange>
+                                        <goals>
+                                            <goal>generate-evaluator</goal>
+                                        </goals>
+                                    </pluginExecutionFilter>
+                                    <action>
+                                        <ignore></ignore>
+                                    </action>
+                                </pluginExecution>
+                                <pluginExecution>
+                                    <pluginExecutionFilter>
                                         <groupId>org.apache.maven.plugins</groupId>
                                         <artifactId>maven-jar-plugin</artifactId>
                                         <versionRange>[2.4,)</versionRange>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/JavaSerializationUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/JavaSerializationUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/JavaSerializationUtils.java
index 1d85fb9..a92e700 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/JavaSerializationUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/JavaSerializationUtils.java
@@ -66,9 +66,8 @@ public class JavaSerializationUtils {
             return null;
         }
         ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
-        try {
-            Thread.currentThread().setContextClassLoader(classLoader);
-            ObjectInputStream ois = new ClassLoaderObjectInputStream(new ByteArrayInputStream(bytes), classLoader);
+        Thread.currentThread().setContextClassLoader(classLoader);
+        try (ObjectInputStream ois = new ClassLoaderObjectInputStream(new ByteArrayInputStream(bytes), classLoader)) {
             return ois.readObject();
         } finally {
             Thread.currentThread().setContextClassLoader(ctxCL);
@@ -82,8 +81,8 @@ public class JavaSerializationUtils {
     private static class ClassLoaderObjectInputStream extends ObjectInputStream {
         private ClassLoader classLoader;
 
-        protected ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException,
-                SecurityException {
+        protected ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader)
+                throws IOException, SecurityException {
             super(in);
             this.classLoader = classLoader;
         }
@@ -99,9 +98,9 @@ public class JavaSerializationUtils {
             boolean hasNonPublicInterface = false;
 
             // define proxy in class loader of non-public interface(s), if any
-            Class[] classObjs = new Class[interfaces.length];
+            Class<?>[] classObjs = new Class[interfaces.length];
             for (int i = 0; i < interfaces.length; i++) {
-                Class cl = Class.forName(interfaces[i], false, classLoader);
+                Class<?> cl = Class.forName(interfaces[i], false, classLoader);
                 if ((cl.getModifiers() & Modifier.PUBLIC) == 0) {
                     if (hasNonPublicInterface) {
                         if (nonPublicLoader != cl.getClassLoader()) {


[2/2] incubator-asterixdb git commit: Bytecode generation for null-handling in SQL++ functions.

Posted by bu...@apache.org.
Bytecode generation for null-handling in SQL++ functions.

Change-Id: I75d8abfad6b0a8c0d7faaac2e7fc7d4fea638e37
Reviewed-on: https://asterix-gerrit.ics.uci.edu/824
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: dbd828587d4454b57a189842c7d947c60b9193aa
Parents: eb2170e
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Fri Apr 22 00:02:39 2016 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Fri Apr 22 17:24:59 2016 -0700

----------------------------------------------------------------------
 asterixdb/asterix-algebra/pom.xml               |  39 +-
 .../AqlPlusExpressionToPlanTranslator.java      |   3 +-
 .../LangExpressionToPlanTranslator.java         |   3 +-
 .../translator/util/FunctionCollection.java     | 604 +++++++++++++++++++
 asterixdb/asterix-app/pom.xml                   |   3 -
 .../queries_sqlpp/null/ceil/ceil.1.query.sqlpp  |  20 +
 .../null/length/length.1.query.sqlpp            |  20 +
 .../null/lower/lower.1.query.sqlpp              |  20 +
 .../null/substr/substr.1.query.sqlpp            |  31 +
 .../null/upper/upper.1.query.sqlpp              |  20 +
 .../runtimets/results/null/ceil/ceil.1.adm      |   1 +
 .../runtimets/results/null/length/length.1.adm  |   1 +
 .../runtimets/results/null/lower/lower.1.adm    |   1 +
 .../runtimets/results/null/substr/substr.1.adm  |   1 +
 .../runtimets/results/null/upper/upper.1.adm    |   1 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  27 +
 .../pom.xml                                     |  68 +++
 .../plugin/EvaluatorGeneratorMojo.java          |  93 +++
 asterixdb/asterix-maven-plugins/pom.xml         |  16 +-
 .../asterix/formats/base/IDataFormat.java       |   3 +-
 asterixdb/asterix-runtime/pom.xml               |  15 +-
 .../functions/NumericCeilingDescriptor.java     |   7 +-
 .../functions/StringLengthDescriptor.java       |   6 -
 .../functions/StringLowerCaseDescriptor.java    |  10 -
 .../functions/StringUpperCaseDescriptor.java    |  10 -
 .../evaluators/staticcodegen/CodeGenUtil.java   | 352 +++++++++++
 .../staticcodegen/EvaluatorVisitor.java         | 125 ++++
 .../GatherEvaluatorCreationVisitor.java         |  67 ++
 .../GatherEvaluatorFactoryCreationVisitor.java  |  66 ++
 .../staticcodegen/GatherInnerClassVisitor.java  |  55 ++
 .../staticcodegen/MethodIdentifier.java         |  57 ++
 .../staticcodegen/RenameClassVisitor.java       | 112 ++++
 .../evaluators/staticcodegen/TypeCheckUtil.java |  41 ++
 .../runtime/formats/NonTaggedDataFormat.java    | 531 +---------------
 asterixdb/pom.xml                               |  13 +
 .../api/util/JavaSerializationUtils.java        |  13 +-
 36 files changed, 1866 insertions(+), 589 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-algebra/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 7bc2b8c..5131852 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -48,6 +48,23 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.asterix</groupId>
+                <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+                <version>${project.version}</version>
+                <configuration>
+                    <evaluatorPackagePrefix>org.apache.asterix.runtime.evaluators</evaluatorPackagePrefix>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>generate-evaluator</id>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>generate-evaluator</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
         <pluginManagement>
             <plugins>
@@ -60,12 +77,22 @@
                             <pluginExecutions>
                                 <pluginExecution>
                                     <pluginExecutionFilter>
-                                        <groupId> org.codehaus.mojo 
-                                        </groupId>
-                                        <artifactId> javacc-maven-plugin 
-                                        </artifactId>
-                                        <versionRange> [2.6,) 
-                                        </versionRange>
+                                        <groupId>org.codehaus.mojo</groupId>
+                                        <artifactId>javacc-maven-plugin</artifactId>
+                                        <versionRange> [2.6,)</versionRange>
+                                        <goals>
+                                            <goal>javacc</goal>
+                                        </goals>
+                                    </pluginExecutionFilter>
+                                    <action>
+                                        <ignore />
+                                    </action>
+                                </pluginExecution>
+                                <pluginExecution>
+                                    <pluginExecutionFilter>
+                                        <groupId>org.apache.asterix</groupId>
+                                        <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+                                        <versionRange>(0.8.8-incubating,)</versionRange>
                                         <goals>
                                             <goal>javacc</goal>
                                         </goals>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 45933f4..d207f20 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -114,6 +114,7 @@ import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.util.FunctionCollection;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -244,7 +245,7 @@ public class AqlPlusExpressionToPlanTranslator extends AbstractLangTranslator
         if (format == null) {
             throw new AlgebricksException("Data format has not been set.");
         }
-        format.registerRuntimeFunctions();
+        format.registerRuntimeFunctions(FunctionCollection.getFunctionDescriptorFactories());
         Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
                 new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator()));
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 9cbc88a..551fa85 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -95,6 +95,7 @@ import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.util.FunctionCollection;
 import org.apache.asterix.translator.util.PlanTranslationUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -173,7 +174,7 @@ class LangExpressionToPlanTranslator
             throws AlgebricksException {
         this.context = new TranslationContext(new Counter(currentVarCounter));
         this.metadataProvider = metadataProvider;
-        FormatUtils.getDefaultFormat().registerRuntimeFunctions();
+        FormatUtils.getDefaultFormat().registerRuntimeFunctions(FunctionCollection.getFunctionDescriptorFactories());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
new file mode 100644
index 0000000..c575476
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
@@ -0,0 +1,604 @@
+/*
+ * 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.translator.util;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.collections.ListifyAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableIntermediateSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableLocalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.MinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.stream.NonEmptyStreamAggregateDescriptor;
+import org.apache.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalDayAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalHourAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndDateAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndDatetimeAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalEndTimeAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartDateAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartDatetimeAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalIntervalStartTimeAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalMillisecondAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalMinuteAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalMonthAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalSecondAccessor;
+import org.apache.asterix.runtime.evaluators.accessors.TemporalYearAccessor;
+import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AFloatConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt16ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt32ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt64ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt8ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AIntervalConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ALineConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ANullConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.APoint3DConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.APointConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.APolygonConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.OpenRecordConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CastListDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CountHashedGramTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CountHashedWordTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateCircleDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateLineDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateMBRDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreatePointDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreatePolygonDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateQueryUIDDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateRectangleDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.CreateUUIDDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.EditDistanceCheckDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.EditDistanceContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.EditDistanceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.EditDistanceListIsFilterable;
+import org.apache.asterix.runtime.evaluators.functions.EditDistanceStringIsFilterable;
+import org.apache.asterix.runtime.evaluators.functions.EmbedTypeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FlowRecordDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FuzzyEqDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.GramTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.HashedGramTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.HashedWordTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.InjectFailureDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.IsNullDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.IsSystemNullDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.LenDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NotNullDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericAddDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericSubDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.OrderedListConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PrefixLenJaccardDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RegExpDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardCheckDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixCheckDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedCheckDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringConcatDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringEqualDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringJoinDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringLengthDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringLikeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringLowerCaseDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringMatchesDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringMatchesWithFlagDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithFlagsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringUpperCaseDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.Substring2Descriptor;
+import org.apache.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SubstringDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.WordTokensDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.BinaryConcatDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.BinaryLengthDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.FindBinaryDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.FindBinaryFromDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.ParseBinaryDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.PrintBinaryDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.SubBinaryFromDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.binary.SubBinaryFromToDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByIndexDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByNameDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessNestedDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.GetRecordFieldValueDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.GetRecordFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordAddFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordMergeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustDateTimeForTimeZoneDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustTimeForTimeZoneDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDuartionFromDateDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDurationFromDateTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromDatetimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromUnixTimeInDaysDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromDateAndTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInMsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInSecsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DayOfWeekDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DayTimeDurationComparatorDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DurationEqualDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromIntervalDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromMillisecondsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DurationFromMonthsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.GetDayTimeDurationDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.GetOverlappingIntervalDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.GetYearMonthDurationDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalAfterDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalBeforeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalBinDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalCoveredByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalCoversDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalEndedByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalEndsDecriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMeetsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMetByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlappedByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapBinsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.ParseTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.PrintTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
+import org.apache.asterix.runtime.evaluators.staticcodegen.CodeGenUtil;
+import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
+import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
+import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
+
+/**
+ * This class (statically) holds a list of function descriptor factories.
+ */
+public class FunctionCollection {
+
+    private static final String FACTORY = "FACTORY";
+    private static final List<IFunctionDescriptorFactory> temp = new ArrayList<IFunctionDescriptorFactory>();
+
+    static {
+        // format-independent
+        temp.add(TidRunningAggregateDescriptor.FACTORY);
+
+        // format-dependent
+        temp.add(AndDescriptor.FACTORY);
+        temp.add(OrDescriptor.FACTORY);
+        temp.add(StringLikeDescriptor.FACTORY);
+        temp.add(ScanCollectionDescriptor.FACTORY);
+        temp.add(AnyCollectionMemberDescriptor.FACTORY);
+        temp.add(ClosedRecordConstructorDescriptor.FACTORY);
+        temp.add(FieldAccessByIndexDescriptor.FACTORY);
+        temp.add(FieldAccessByNameDescriptor.FACTORY);
+        temp.add(FieldAccessNestedDescriptor.FACTORY);
+        temp.add(GetRecordFieldsDescriptor.FACTORY);
+        temp.add(GetRecordFieldValueDescriptor.FACTORY);
+        temp.add(FieldAccessByNameDescriptor.FACTORY);
+        temp.add(GetItemDescriptor.FACTORY);
+        temp.add(NumericUnaryMinusDescriptor.FACTORY);
+        temp.add(OpenRecordConstructorDescriptor.FACTORY);
+        temp.add(OrderedListConstructorDescriptor.FACTORY);
+        temp.add(UnorderedListConstructorDescriptor.FACTORY);
+        temp.add(EmbedTypeDescriptor.FACTORY);
+
+        temp.add(NumericAddDescriptor.FACTORY);
+        temp.add(NumericDivideDescriptor.FACTORY);
+        temp.add(NumericMultiplyDescriptor.FACTORY);
+        temp.add(NumericSubDescriptor.FACTORY);
+        temp.add(NumericModuloDescriptor.FACTORY);
+        temp.add(NumericCaretDescriptor.FACTORY);
+        temp.add(IsNullDescriptor.FACTORY);
+        temp.add(IsSystemNullDescriptor.FACTORY);
+        temp.add(NotDescriptor.FACTORY);
+        temp.add(LenDescriptor.FACTORY);
+        temp.add(EmptyStreamAggregateDescriptor.FACTORY);
+        temp.add(NonEmptyStreamAggregateDescriptor.FACTORY);
+        temp.add(RangeDescriptor.FACTORY);
+
+        temp.add(NumericAbsDescriptor.FACTORY);
+        temp.add(getGeneratedFunctionDescriptorFactory(NumericCeilingDescriptor.class));
+        temp.add(NumericFloorDescriptor.FACTORY);
+        temp.add(NumericRoundDescriptor.FACTORY);
+        temp.add(NumericRoundHalfToEvenDescriptor.FACTORY);
+        temp.add(NumericRoundHalfToEven2Descriptor.FACTORY);
+
+        // Binary functions
+        temp.add(BinaryLengthDescriptor.FACTORY);
+        temp.add(ParseBinaryDescriptor.FACTORY);
+        temp.add(PrintBinaryDescriptor.FACTORY);
+        temp.add(BinaryConcatDescriptor.FACTORY);
+        temp.add(SubBinaryFromDescriptor.FACTORY);
+        temp.add(SubBinaryFromToDescriptor.FACTORY);
+        temp.add(FindBinaryDescriptor.FACTORY);
+        temp.add(FindBinaryFromDescriptor.FACTORY);
+
+        // String functions
+        temp.add(StringContainsDescriptor.FACTORY);
+        temp.add(StringEndsWithDescriptor.FACTORY);
+        temp.add(StringStartsWithDescriptor.FACTORY);
+        temp.add(getGeneratedFunctionDescriptorFactory(SubstringDescriptor.class));
+        temp.add(StringEqualDescriptor.FACTORY);
+        temp.add(StringMatchesDescriptor.FACTORY);
+        temp.add(getGeneratedFunctionDescriptorFactory(StringLowerCaseDescriptor.class));
+        temp.add(getGeneratedFunctionDescriptorFactory(StringUpperCaseDescriptor.class));
+        temp.add(StringMatchesWithFlagDescriptor.FACTORY);
+        temp.add(StringReplaceDescriptor.FACTORY);
+        temp.add(StringReplaceWithFlagsDescriptor.FACTORY);
+        temp.add(getGeneratedFunctionDescriptorFactory(StringLengthDescriptor.class));
+        temp.add(getGeneratedFunctionDescriptorFactory(Substring2Descriptor.class));
+        temp.add(SubstringBeforeDescriptor.FACTORY);
+        temp.add(SubstringAfterDescriptor.FACTORY);
+        temp.add(StringToCodePointDescriptor.FACTORY);
+        temp.add(CodePointToStringDescriptor.FACTORY);
+        temp.add(StringConcatDescriptor.FACTORY);
+        temp.add(StringJoinDescriptor.FACTORY);
+
+        // aggregates
+        temp.add(ListifyAggregateDescriptor.FACTORY);
+        temp.add(CountAggregateDescriptor.FACTORY);
+        temp.add(AvgAggregateDescriptor.FACTORY);
+        temp.add(LocalAvgAggregateDescriptor.FACTORY);
+        temp.add(IntermediateAvgAggregateDescriptor.FACTORY);
+        temp.add(GlobalAvgAggregateDescriptor.FACTORY);
+        temp.add(SumAggregateDescriptor.FACTORY);
+        temp.add(LocalSumAggregateDescriptor.FACTORY);
+        temp.add(MaxAggregateDescriptor.FACTORY);
+        temp.add(LocalMaxAggregateDescriptor.FACTORY);
+        temp.add(MinAggregateDescriptor.FACTORY);
+        temp.add(LocalMinAggregateDescriptor.FACTORY);
+
+        // serializable aggregates
+        temp.add(SerializableCountAggregateDescriptor.FACTORY);
+        temp.add(SerializableAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableLocalAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableIntermediateAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableGlobalAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableSumAggregateDescriptor.FACTORY);
+        temp.add(SerializableLocalSumAggregateDescriptor.FACTORY);
+
+        // scalar aggregates
+        temp.add(ScalarCountAggregateDescriptor.FACTORY);
+        temp.add(ScalarAvgAggregateDescriptor.FACTORY);
+        temp.add(ScalarSumAggregateDescriptor.FACTORY);
+        temp.add(ScalarMaxAggregateDescriptor.FACTORY);
+        temp.add(ScalarMinAggregateDescriptor.FACTORY);
+
+        // SQL aggregates
+        temp.add(SqlCountAggregateDescriptor.FACTORY);
+        temp.add(SqlAvgAggregateDescriptor.FACTORY);
+        temp.add(LocalSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(IntermediateSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(GlobalSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(SqlSumAggregateDescriptor.FACTORY);
+        temp.add(LocalSqlSumAggregateDescriptor.FACTORY);
+        temp.add(SqlMaxAggregateDescriptor.FACTORY);
+        temp.add(LocalSqlMaxAggregateDescriptor.FACTORY);
+        temp.add(SqlMinAggregateDescriptor.FACTORY);
+        temp.add(LocalSqlMinAggregateDescriptor.FACTORY);
+
+        // SQL serializable aggregates
+        temp.add(SerializableSqlCountAggregateDescriptor.FACTORY);
+        temp.add(SerializableSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableLocalSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableIntermediateSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableGlobalSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(SerializableSqlSumAggregateDescriptor.FACTORY);
+        temp.add(SerializableLocalSqlSumAggregateDescriptor.FACTORY);
+
+        // SQL scalar aggregates
+        temp.add(ScalarSqlCountAggregateDescriptor.FACTORY);
+        temp.add(ScalarSqlAvgAggregateDescriptor.FACTORY);
+        temp.add(ScalarSqlSumAggregateDescriptor.FACTORY);
+        temp.add(ScalarSqlMaxAggregateDescriptor.FACTORY);
+        temp.add(ScalarSqlMinAggregateDescriptor.FACTORY);
+
+        // new functions - constructors
+        temp.add(ABooleanConstructorDescriptor.FACTORY);
+        temp.add(ANullConstructorDescriptor.FACTORY);
+        temp.add(ABinaryHexStringConstructorDescriptor.FACTORY);
+        temp.add(ABinaryBase64StringConstructorDescriptor.FACTORY);
+        temp.add(AStringConstructorDescriptor.FACTORY);
+        temp.add(AInt8ConstructorDescriptor.FACTORY);
+        temp.add(AInt16ConstructorDescriptor.FACTORY);
+        temp.add(AInt32ConstructorDescriptor.FACTORY);
+        temp.add(AInt64ConstructorDescriptor.FACTORY);
+        temp.add(AFloatConstructorDescriptor.FACTORY);
+        temp.add(ADoubleConstructorDescriptor.FACTORY);
+        temp.add(APointConstructorDescriptor.FACTORY);
+        temp.add(APoint3DConstructorDescriptor.FACTORY);
+        temp.add(ALineConstructorDescriptor.FACTORY);
+        temp.add(APolygonConstructorDescriptor.FACTORY);
+        temp.add(ACircleConstructorDescriptor.FACTORY);
+        temp.add(ARectangleConstructorDescriptor.FACTORY);
+        temp.add(ATimeConstructorDescriptor.FACTORY);
+        temp.add(ADateConstructorDescriptor.FACTORY);
+        temp.add(ADateTimeConstructorDescriptor.FACTORY);
+        temp.add(ADurationConstructorDescriptor.FACTORY);
+        temp.add(AYearMonthDurationConstructorDescriptor.FACTORY);
+        temp.add(ADayTimeDurationConstructorDescriptor.FACTORY);
+        temp.add(AUUIDFromStringConstructorDescriptor.FACTORY);
+
+        temp.add(DeepEqualityDescriptor.FACTORY);
+
+        temp.add(CreateUUIDDescriptor.FACTORY);
+        temp.add(CreateQueryUIDDescriptor.FACTORY);
+        // Spatial
+        temp.add(CreatePointDescriptor.FACTORY);
+        temp.add(CreateLineDescriptor.FACTORY);
+        temp.add(CreatePolygonDescriptor.FACTORY);
+        temp.add(CreateCircleDescriptor.FACTORY);
+        temp.add(CreateRectangleDescriptor.FACTORY);
+        temp.add(SpatialAreaDescriptor.FACTORY);
+        temp.add(SpatialDistanceDescriptor.FACTORY);
+        temp.add(SpatialIntersectDescriptor.FACTORY);
+        temp.add(CreateMBRDescriptor.FACTORY);
+        temp.add(SpatialCellDescriptor.FACTORY);
+        temp.add(PointXCoordinateAccessor.FACTORY);
+        temp.add(PointYCoordinateAccessor.FACTORY);
+        temp.add(CircleRadiusAccessor.FACTORY);
+        temp.add(CircleCenterAccessor.FACTORY);
+        temp.add(LineRectanglePolygonAccessor.FACTORY);
+
+        // fuzzyjoin function
+        temp.add(FuzzyEqDescriptor.FACTORY);
+        temp.add(SubsetCollectionDescriptor.FACTORY);
+        temp.add(PrefixLenJaccardDescriptor.FACTORY);
+
+        temp.add(WordTokensDescriptor.FACTORY);
+        temp.add(HashedWordTokensDescriptor.FACTORY);
+        temp.add(CountHashedWordTokensDescriptor.FACTORY);
+
+        temp.add(GramTokensDescriptor.FACTORY);
+        temp.add(HashedGramTokensDescriptor.FACTORY);
+        temp.add(CountHashedGramTokensDescriptor.FACTORY);
+
+        temp.add(EditDistanceDescriptor.FACTORY);
+        temp.add(EditDistanceCheckDescriptor.FACTORY);
+        temp.add(EditDistanceStringIsFilterable.FACTORY);
+        temp.add(EditDistanceListIsFilterable.FACTORY);
+        temp.add(EditDistanceContainsDescriptor.FACTORY);
+
+        temp.add(SimilarityJaccardDescriptor.FACTORY);
+        temp.add(SimilarityJaccardCheckDescriptor.FACTORY);
+        temp.add(SimilarityJaccardSortedDescriptor.FACTORY);
+        temp.add(SimilarityJaccardSortedCheckDescriptor.FACTORY);
+        temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
+        temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
+
+        //Record functions
+        temp.add(RecordMergeDescriptor.FACTORY);
+        temp.add(RecordAddFieldsDescriptor.FACTORY);
+        temp.add(RecordRemoveFieldsDescriptor.FACTORY);
+        temp.add(SwitchCaseDescriptor.FACTORY);
+        temp.add(RegExpDescriptor.FACTORY);
+        temp.add(InjectFailureDescriptor.FACTORY);
+        temp.add(CastListDescriptor.FACTORY);
+        temp.add(CastRecordDescriptor.FACTORY);
+        temp.add(FlowRecordDescriptor.FACTORY);
+        temp.add(NotNullDescriptor.FACTORY);
+
+        // Spatial and temporal type accessors
+        temp.add(TemporalYearAccessor.FACTORY);
+        temp.add(TemporalMonthAccessor.FACTORY);
+        temp.add(TemporalDayAccessor.FACTORY);
+        temp.add(TemporalHourAccessor.FACTORY);
+        temp.add(TemporalMinuteAccessor.FACTORY);
+        temp.add(TemporalSecondAccessor.FACTORY);
+        temp.add(TemporalMillisecondAccessor.FACTORY);
+        temp.add(TemporalIntervalStartAccessor.FACTORY);
+        temp.add(TemporalIntervalEndAccessor.FACTORY);
+        temp.add(TemporalIntervalStartDateAccessor.FACTORY);
+        temp.add(TemporalIntervalEndDateAccessor.FACTORY);
+        temp.add(TemporalIntervalStartTimeAccessor.FACTORY);
+        temp.add(TemporalIntervalEndTimeAccessor.FACTORY);
+        temp.add(TemporalIntervalStartDatetimeAccessor.FACTORY);
+        temp.add(TemporalIntervalEndDatetimeAccessor.FACTORY);
+
+        // Temporal functions
+        temp.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
+        temp.add(DateFromDatetimeDescriptor.FACTORY);
+        temp.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
+        temp.add(TimeFromDatetimeDescriptor.FACTORY);
+        temp.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
+        temp.add(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
+        temp.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
+        temp.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
+        temp.add(CalendarDuartionFromDateDescriptor.FACTORY);
+        temp.add(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
+        temp.add(AdjustTimeForTimeZoneDescriptor.FACTORY);
+        temp.add(IntervalBeforeDescriptor.FACTORY);
+        temp.add(IntervalAfterDescriptor.FACTORY);
+        temp.add(IntervalMeetsDescriptor.FACTORY);
+        temp.add(IntervalMetByDescriptor.FACTORY);
+        temp.add(IntervalOverlapsDescriptor.FACTORY);
+        temp.add(IntervalOverlappedByDescriptor.FACTORY);
+        temp.add(OverlapDescriptor.FACTORY);
+        temp.add(IntervalStartsDescriptor.FACTORY);
+        temp.add(IntervalStartedByDescriptor.FACTORY);
+        temp.add(IntervalCoversDescriptor.FACTORY);
+        temp.add(IntervalCoveredByDescriptor.FACTORY);
+        temp.add(IntervalEndsDecriptor.FACTORY);
+        temp.add(IntervalEndedByDescriptor.FACTORY);
+        temp.add(CurrentDateDescriptor.FACTORY);
+        temp.add(CurrentTimeDescriptor.FACTORY);
+        temp.add(CurrentDateTimeDescriptor.FACTORY);
+        temp.add(DurationFromMillisecondsDescriptor.FACTORY);
+        temp.add(DurationFromMonthsDescriptor.FACTORY);
+        temp.add(YearMonthDurationComparatorDecriptor.GREATER_THAN_FACTORY);
+        temp.add(YearMonthDurationComparatorDecriptor.LESS_THAN_FACTORY);
+        temp.add(DayTimeDurationComparatorDescriptor.GREATER_THAN_FACTORY);
+        temp.add(DayTimeDurationComparatorDescriptor.LESS_THAN_FACTORY);
+        temp.add(MonthsFromYearMonthDurationDescriptor.FACTORY);
+        temp.add(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
+        temp.add(DurationEqualDescriptor.FACTORY);
+        temp.add(GetYearMonthDurationDescriptor.FACTORY);
+        temp.add(GetDayTimeDurationDescriptor.FACTORY);
+        temp.add(IntervalBinDescriptor.FACTORY);
+        temp.add(OverlapBinsDescriptor.FACTORY);
+        temp.add(DayOfWeekDescriptor.FACTORY);
+        temp.add(ParseDateDescriptor.FACTORY);
+        temp.add(ParseTimeDescriptor.FACTORY);
+        temp.add(ParseDateTimeDescriptor.FACTORY);
+        temp.add(PrintDateDescriptor.FACTORY);
+        temp.add(PrintTimeDescriptor.FACTORY);
+        temp.add(PrintDateTimeDescriptor.FACTORY);
+        temp.add(GetOverlappingIntervalDescriptor.FACTORY);
+        temp.add(DurationFromIntervalDescriptor.FACTORY);
+
+        // Interval constructor
+        temp.add(AIntervalConstructorDescriptor.FACTORY);
+        temp.add(AIntervalStartFromDateConstructorDescriptor.FACTORY);
+        temp.add(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
+        temp.add(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
+    }
+
+    public static List<IFunctionDescriptorFactory> getFunctionDescriptorFactories() {
+        return temp;
+    }
+
+    /**
+     * Gets the generated function descriptor factory from an <code>IFunctionDescriptor</code>
+     * implementation class.
+     *
+     * @param cl,
+     *            the class of an <code>IFunctionDescriptor</code> implementation.
+     * @return the IFunctionDescriptorFactory instance defined in the class.
+     */
+    private static IFunctionDescriptorFactory getGeneratedFunctionDescriptorFactory(Class<?> cl) {
+        try {
+            String className = cl.getName() + CodeGenUtil.DEFAULT_SUFFIX_FOR_GENERATED_CLASS;
+            Class<?> generatedCl = cl.getClassLoader().loadClass(className);
+            Field factory = generatedCl.getDeclaredField(FACTORY);
+            return (IFunctionDescriptorFactory) factory.get(null);
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index d8e7aec..5ca06b6 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -185,9 +185,6 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
-		<!-- posssible remove this <dependency> <groupId>com.kenai.nbpwr</groupId> 
-			<artifactId>org-apache-commons-io</artifactId> <version>1.3.1-201002241208</version> 
-			<scope>test</scope> </dependency> -->
         <dependency>
             <groupId>org.apache.asterix</groupId>
             <artifactId>asterix-transactions</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/ceil/ceil.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/ceil/ceil.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/ceil/ceil.1.query.sqlpp
new file mode 100644
index 0000000..c880e2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/ceil/ceil.1.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+CEIL(null);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/length/length.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/length/length.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/length/length.1.query.sqlpp
new file mode 100644
index 0000000..034ea1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/length/length.1.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+LENGTH(null);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/lower/lower.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/lower/lower.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/lower/lower.1.query.sqlpp
new file mode 100644
index 0000000..fbf98ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/lower/lower.1.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+LOWER(null);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/substr/substr.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/substr/substr.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/substr/substr.1.query.sqlpp
new file mode 100644
index 0000000..bece63a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/substr/substr.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  'a': SUBSTR(null, 1),
+  'b': SUBSTR('abc', null),
+  'c': SUBSTR(null, null),
+  'd': SUBSTR(null, 1, 2),
+  'e': substr('abc', null, 2),
+  'f': SUBSTR('abc', 1, null),
+  'g': SUBSTR(null, null, 2),
+  'h': substr(null, 1, null),
+  'i': SUBSTR(null, null, null),
+  'j': SUBSTR('abc', null, null)
+};

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/upper/upper.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/upper/upper.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/upper/upper.1.query.sqlpp
new file mode 100644
index 0000000..367ee62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null/upper/upper.1.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+UPPER(null);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/results/null/ceil/ceil.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null/ceil/ceil.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/ceil/ceil.1.adm
new file mode 100644
index 0000000..ec747fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/ceil/ceil.1.adm
@@ -0,0 +1 @@
+null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/results/null/length/length.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null/length/length.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/length/length.1.adm
new file mode 100644
index 0000000..ec747fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/length/length.1.adm
@@ -0,0 +1 @@
+null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/results/null/lower/lower.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null/lower/lower.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/lower/lower.1.adm
new file mode 100644
index 0000000..ec747fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/lower/lower.1.adm
@@ -0,0 +1 @@
+null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/results/null/substr/substr.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null/substr/substr.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/substr/substr.1.adm
new file mode 100644
index 0000000..92f6a06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/substr/substr.1.adm
@@ -0,0 +1 @@
+{ "a": null, "b": null, "c": null, "d": null, "e": null, "f": null, "g": null, "h": null, "i": null, "j": null }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/results/null/upper/upper.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null/upper/upper.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/upper/upper.1.adm
new file mode 100644
index 0000000..ec747fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null/upper/upper.1.adm
@@ -0,0 +1 @@
+null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index f9d1a0e..70828f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3582,6 +3582,33 @@
             </compilation-unit>
         </test-case>
     </test-group>
+    <test-group name="null">
+        <test-case FilePath="null">
+            <compilation-unit name="ceil">
+                <output-dir compare="Text">ceil</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="null">
+            <compilation-unit name="length">
+                <output-dir compare="Text">length</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="null">
+            <compilation-unit name="lower">
+                <output-dir compare="Text">lower</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="null">
+            <compilation-unit name="substr">
+                <output-dir compare="Text">substr</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="null">
+            <compilation-unit name="upper">
+                <output-dir compare="Text">upper</output-dir>
+            </compilation-unit>
+        </test-case>
+    </test-group>
     <test-group name="numeric">
         <test-case FilePath="numeric">
             <compilation-unit name="caret0">

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
new file mode 100644
index 0000000..9d4a963
--- /dev/null
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
@@ -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.
+ !-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>org.apache.asterix</groupId>
+  <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+  <parent>
+        <groupId>org.apache.asterix</groupId>
+        <artifactId>asterix-maven-plugins</artifactId>
+        <version>0.8.9-SNAPSHOT</version>
+  </parent>
+
+  <packaging>maven-plugin</packaging>
+  <name>asterix-evaluator-generator-maven-plugin</name>
+
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+  <dependencies>
+    <dependency>
+       <groupId>org.apache.asterix</groupId>
+       <artifactId>asterix-runtime</artifactId>
+       <type>jar</type>
+       <version>${project.version}</version>
+       <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.maven</groupId>
+      <artifactId>maven-plugin-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.maven</groupId>
+      <artifactId>maven-artifact</artifactId>
+      <version>2.2.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.maven</groupId>
+      <artifactId>maven-project</artifactId>
+      <version>2.2.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.reflections</groupId>
+      <artifactId>reflections</artifactId>
+      <version>0.9.10</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/plugin/EvaluatorGeneratorMojo.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/plugin/EvaluatorGeneratorMojo.java b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/plugin/EvaluatorGeneratorMojo.java
new file mode 100644
index 0000000..d001bb0
--- /dev/null
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/plugin/EvaluatorGeneratorMojo.java
@@ -0,0 +1,93 @@
+/*
+ * 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.plugin;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.staticcodegen.CodeGenUtil;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.project.MavenProject;
+import org.reflections.Reflections;
+
+/**
+ * Statically generates null-handling byte code for scalar functions.
+ *
+ * @goal generate-evaluator
+ * @phase compile
+ */
+public class EvaluatorGeneratorMojo extends AbstractMojo {
+
+    private String baseDir;
+
+    /**
+     * @parameter default-value="${project}"
+     * @required
+     * @readonly
+     */
+    MavenProject project;
+
+    /**
+     * @parameter default-value="${evaluatorPackagePrefix}"
+     * @required
+     * @readonly
+     */
+    String evaluatorPackagePrefix;
+
+    public EvaluatorGeneratorMojo() {
+    }
+
+    @Override
+    public void execute() throws MojoExecutionException, MojoFailureException {
+        baseDir = project.getBuild().getDirectory() + File.separator + "classes";
+        try {
+            // Finds all sub-classes of AbstractScalarFunctionDynamicDescriptor with in the package
+            // org.apache.asterix.runtime.evaluators.
+            Reflections reflections = new Reflections(evaluatorPackagePrefix);
+            Set<Class<? extends AbstractScalarFunctionDynamicDescriptor>> allClasses = reflections
+                    .getSubTypesOf(AbstractScalarFunctionDynamicDescriptor.class);
+
+            // Generates byte code for all sub-classes of AbstractScalarFunctionDynamicDescriptor.
+            for (Class<?> cl : allClasses) {
+                getLog().info("Generating byte code for " + cl.getName());
+                CodeGenUtil.generateScalarFunctionDescriptorBinary(evaluatorPackagePrefix, cl.getName(),
+                        CodeGenUtil.DEFAULT_SUFFIX_FOR_GENERATED_CLASS, reflections.getClass().getClassLoader(),
+                        (name, bytes) -> writeFile(name, bytes));
+            }
+        } catch (Exception e) {
+            getLog().error(e);
+            throw new MojoFailureException(e.toString());
+        }
+    }
+
+    private void writeFile(String name, byte[] classDefinitionBinary) throws IOException {
+        File targetFile = new File(baseDir + File.separator + name + ".class");
+        targetFile.getParentFile().mkdirs();
+        targetFile.createNewFile();
+        try (FileOutputStream outputStream = new FileOutputStream(targetFile)) {
+            outputStream.write(classDefinitionBinary, 0, classDefinitionBinary.length);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-maven-plugins/pom.xml b/asterixdb/asterix-maven-plugins/pom.xml
index c4a6ff9..0737ccb 100644
--- a/asterixdb/asterix-maven-plugins/pom.xml
+++ b/asterixdb/asterix-maven-plugins/pom.xml
@@ -1,3 +1,4 @@
+<?xml version="1.0" encoding="UTF-8"?>
 <!--
  ! Licensed to the Apache Software Foundation (ASF) under one
  ! or more contributor license agreements.  See the NOTICE file
@@ -43,16 +44,17 @@
 	</parent>
 
   <dependencies>
-  	<dependency>
-  		<groupId>org.apache.maven</groupId>
-  		<artifactId>maven-plugin-api</artifactId>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
+    <dependency>
+        <groupId>org.apache.maven</groupId>
+        <artifactId>maven-plugin-api</artifactId>
+        <type>jar</type>
+        <scope>compile</scope>
+    </dependency>
   </dependencies>
 
   <modules>
     <module>lexer-generator-maven-plugin</module>
     <module>record-manager-generator-maven-plugin</module>
+    <module>asterix-evaluator-generator-maven-plugin</module>
   </modules>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 01897d4..f87681f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -21,6 +21,7 @@ package org.apache.asterix.formats.base;
 import java.util.List;
 
 import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -44,7 +45,7 @@ import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 
 public interface IDataFormat {
-    public void registerRuntimeFunctions() throws AlgebricksException;
+    public void registerRuntimeFunctions(List<IFunctionDescriptorFactory> funcDescriptor) throws AlgebricksException;
 
     public ISerializerDeserializerProvider getSerdeProvider();
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 1914336..6ead8f3 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -53,6 +53,10 @@
             <artifactId>hyracks-storage-am-btree</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-api</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.asterix</groupId>
             <artifactId>asterix-transactions</artifactId>
             <version>0.8.9-SNAPSHOT</version>
@@ -63,10 +67,17 @@
             <artifactId>hadoop-client</artifactId>
             <type>jar</type>
             <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>asm</groupId>
+                    <artifactId>asm</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
-            <groupId>org.apache.hyracks</groupId>
-            <artifactId>hyracks-api</artifactId>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm-all</artifactId>
+            <version>5.1</version>
         </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
index 0a42ffb..89854fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -38,7 +38,6 @@ import org.apache.asterix.om.base.AMutableInt16;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.AMutableInt8;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -106,11 +105,7 @@ public class NumericCeilingDescriptor extends AbstractScalarFunctionDynamicDescr
                         int offset = argPtr.getStartOffset();
 
                         try {
-                            if (data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ANULL);
-                                serde.serialize(ANull.NULL, out);
-                            } else if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
+                            if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
                                 serde = AqlSerializerDeserializerProvider.INSTANCE
                                         .getSerializerDeserializer(BuiltinType.AINT8);
                                 byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index 6a38bdb..f1a0a4f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 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;
@@ -67,9 +66,6 @@ public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescrip
                     private IPointable inputArg = new VoidPointable();
                     private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
                     @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
-                    @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.AINT64);
 
@@ -86,8 +82,6 @@ public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescrip
                                 int len = UTF8StringUtil.getUTFLength(serString, offset + 1);
                                 result.setValue(len);
                                 int64Serde.serialize(result, out);
-                            } else if (serString[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, out);
                             } else {
                                 throw new AlgebricksException(AsterixBuiltinFunctions.STRING_LENGTH.getName()
                                         + ": expects input type STRING/NULL but got "

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index c025947..341a26a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -21,13 +21,10 @@ 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.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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -35,7 +32,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.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -72,10 +68,6 @@ public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDesc
                     private final UTF8StringBuilder builder = new UTF8StringBuilder();
                     private final UTF8StringPointable string = new UTF8StringPointable();
 
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
-
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
 
@@ -93,8 +85,6 @@ public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDesc
 
                                 out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                                 out.write(array.getByteArray(), 0, array.getLength());
-                            } else if (serString[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, out);
                             } else {
                                 throw new AlgebricksException(AsterixBuiltinFunctions.STRING_LOWERCASE.getName()
                                         + ": expects input type STRING/NULL but got "

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/dbd82858/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
index 473ee94..9676b8f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
@@ -21,13 +21,10 @@ 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.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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -35,7 +32,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.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -74,10 +70,6 @@ public class StringUpperCaseDescriptor extends AbstractScalarFunctionDynamicDesc
                     private final UTF8StringBuilder builder = new UTF8StringBuilder();
                     private final UTF8StringPointable string = new UTF8StringPointable();
 
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
-
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
                         try {
@@ -93,8 +85,6 @@ public class StringUpperCaseDescriptor extends AbstractScalarFunctionDynamicDesc
 
                                 out.writeByte(stt);
                                 out.write(array.getByteArray(), 0, array.getLength());
-                            } else if (serString[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, out);
                             } else {
                                 throw new AlgebricksException(AsterixBuiltinFunctions.STRING_UPPERCASE.getName()
                                         + ": expects input type STRING/NULL but got "