You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org> on 2017/11/01 01:33:47 UTC

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Dmitry Lychagin has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/2118

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................

[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
58 files changed, 1,189 insertions(+), 976 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
index 4803691..fb74475 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.common.api.IExtension;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.om.functions.IFunctionManager;
 
 /**
  * An interface for language extensions
@@ -38,4 +39,8 @@
     }
 
     ILangCompilationProvider getLangCompilationProvider(Language lang);
+
+    default IFunctionManager getFunctionManager() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 9f88bb6..f0d2355 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -23,10 +23,12 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
-import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -40,6 +42,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -51,9 +54,10 @@
 
 public class QueryLogicalExpressionJobGen implements ILogicalExpressionJobGen {
 
-    public static final QueryLogicalExpressionJobGen INSTANCE = new QueryLogicalExpressionJobGen();
+    private final IFunctionManager functionManager;
 
-    private QueryLogicalExpressionJobGen() {
+    public QueryLogicalExpressionJobGen(IFunctionManager functionManager) {
+        this.functionManager = functionManager;
     }
 
     @Override
@@ -61,7 +65,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
+        IFunctionDescriptor fd = resolveFunction(expr, env);
         switch (fd.getFunctionDescriptorTag()) {
             case SERIALAGGREGATE:
                 return null;
@@ -79,7 +83,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return getFunctionDescriptor(expr, env, context).createRunningAggregateEvaluatorFactory(args);
+        return resolveFunction(expr, env).createRunningAggregateEvaluatorFactory(args);
     }
 
     @Override
@@ -87,22 +91,22 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return getFunctionDescriptor(expr, env, context).createUnnestingEvaluatorFactory(args);
+        return resolveFunction(expr, env).createUnnestingEvaluatorFactory(args);
     }
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
             IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        IScalarEvaluatorFactory copyEvaluatorFactory = null;
+        IScalarEvaluatorFactory copyEvaluatorFactory;
         switch (expr.getExpressionTag()) {
             case VARIABLE: {
                 VariableReferenceExpression v = (VariableReferenceExpression) expr;
-                copyEvaluatorFactory = createVariableEvaluatorFactory(v, inputSchemas, context);
+                copyEvaluatorFactory = createVariableEvaluatorFactory(v, inputSchemas);
                 return copyEvaluatorFactory;
             }
             case CONSTANT: {
                 ConstantExpression c = (ConstantExpression) expr;
-                copyEvaluatorFactory = createConstantEvaluatorFactory(c, inputSchemas, context);
+                copyEvaluatorFactory = createConstantEvaluatorFactory(c, context);
                 return copyEvaluatorFactory;
             }
             case FUNCTION_CALL: {
@@ -117,7 +121,7 @@
     }
 
     private IScalarEvaluatorFactory createVariableEvaluatorFactory(VariableReferenceExpression expr,
-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+            IOperatorSchema[] inputSchemas) throws AlgebricksException {
         LogicalVariable variable = expr.getVariableReference();
         for (IOperatorSchema scm : inputSchemas) {
             int pos = scm.findVariable(variable);
@@ -132,22 +136,17 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = null;
-        if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
-            IDataFormat format = FormatUtils.getDefaultFormat();
-            fd = format.resolveFunction(expr, env);
-        } else {
-            ICcApplicationContext appCtx = (ICcApplicationContext) context.getAppContext();
-            fd = ExternalFunctionDescriptorProvider
-                    .getExternalFunctionDescriptor((IExternalFunctionInfo) expr.getFunctionInfo(), appCtx);
-        }
+        IFunctionDescriptor fd = expr.getFunctionInfo() instanceof IExternalFunctionInfo
+                ? ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor(
+                        (IExternalFunctionInfo) expr.getFunctionInfo(), (ICcApplicationContext) context.getAppContext())
+                : resolveFunction(expr, env);
         return fd.createEvaluatorFactory(args);
     }
 
-    private IScalarEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr,
-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        IDataFormat format = FormatUtils.getDefaultFormat();
-        return format.getConstantEvalFactory(expr.getValue());
+    private IScalarEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr, JobGenContext context)
+            throws AlgebricksException {
+        MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+        return metadataProvider.getFormat().getConstantEvalFactory(expr.getValue());
     }
 
     private IScalarEvaluatorFactory[] codegenArguments(AbstractFunctionCallExpression expr,
@@ -168,14 +167,14 @@
             AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
             JobGenContext context) throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
+        IFunctionDescriptor fd = resolveFunction(expr, env);
 
         switch (fd.getFunctionDescriptorTag()) {
             case AGGREGATE: {
                 if (BuiltinFunctions.isAggregateFunctionSerializable(fd.getIdentifier())) {
                     AggregateFunctionCallExpression serialAggExpr = BuiltinFunctions
                             .makeSerializableAggregateFunctionExpression(fd.getIdentifier(), expr.getArguments());
-                    IFunctionDescriptor afdd = getFunctionDescriptor(serialAggExpr, env, context);
+                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env);
                     return afdd.createSerializableAggregateEvaluatorFactory(args);
                 } else {
                     throw new AlgebricksException(
@@ -194,10 +193,14 @@
         }
     }
 
-    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
-            JobGenContext context) throws AlgebricksException {
-        IFunctionDescriptor fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
+    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env)
+            throws AlgebricksException {
+        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
+        IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
+        IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
+        if (fnTypeInfer != null) {
+            fnTypeInfer.infer(expr, fd, env);
+        }
         return fd;
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 5ef41c4..f7695ea 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -36,9 +36,11 @@
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -125,12 +127,13 @@
     private static final IOperatorSchema[] _emptySchemas = new IOperatorSchema[] {};
 
     public ConstantFoldingRule(ICcApplicationContext appCtx) {
-        jobGenCtx = new JobGenContext(null, null, appCtx, SerializerDeserializerProvider.INSTANCE,
+        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
                 BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
                 BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
                 BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, null,
-                new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE), ExpressionTypeComputer.INSTANCE,
-                null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
+                ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 8290446..3ab5611 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -321,7 +321,8 @@
         builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
         builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
         builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE));
+        builder.setExpressionRuntimeProvider(
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
         builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
         builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
         builder.setMissingWriterFactory(format.getMissingWriterFactory());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 0c6b2cc..768416d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -18,9 +18,9 @@
  */
 package org.apache.asterix.app.cc;
 
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.asterix.algebra.base.ILangExtension;
@@ -35,22 +35,25 @@
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.runtime.functions.FunctionCollection;
+import org.apache.asterix.runtime.functions.FunctionManager;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.utils.ExtensionUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * AsterixDB's implementation of {@code IAlgebraExtensionManager} which takes care of
- * initializing extensions for App and Compilation purposes
+ * AsterixDB's implementation of {@code IAlgebraExtensionManager} and {@code IFunctionExtensionManager}
+ * which takes care of initializing extensions for App and Compilation purposes
  */
-public class CCExtensionManager implements IAlgebraExtensionManager {
-
-    private final Map<ExtensionId, IExtension> extensions = new HashMap<>();
+public class CCExtensionManager implements IAlgebraExtensionManager, IFunctionExtensionManager {
 
     private final IStatementExecutorExtension statementExecutorExtension;
     private final ILangCompilationProvider aqlCompilationProvider;
     private final ILangCompilationProvider sqlppCompilationProvider;
+    private final IFunctionManager functionManager;
     private transient IStatementExecutorFactory statementExecutorFactory;
 
     /**
@@ -67,15 +70,16 @@
             throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
         Pair<ExtensionId, ILangCompilationProvider> aqlcp = null;
         Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
+        Pair<ExtensionId, IFunctionManager> fm = null;
         IStatementExecutorExtension see = null;
         if (list != null) {
+            Set<ExtensionId> extensionIds = new HashSet<>();
             for (AsterixExtension extensionConf : list) {
                 IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
                 extension.configure(extensionConf.getArgs());
-                if (extensions.containsKey(extension.getId())) {
+                if (!extensionIds.add(extension.getId())) {
                     throw new RuntimeDataException(ErrorCode.EXTENSION_ID_CONFLICT, extension.getId());
                 }
-                extensions.put(extension.getId(), extension);
                 switch (extension.getExtensionKind()) {
                     case STATEMENT_EXECUTOR:
                         see = ExtensionUtil.extendStatementExecutor(see, (IStatementExecutorExtension) extension);
@@ -84,6 +88,7 @@
                         ILangExtension le = (ILangExtension) extension;
                         aqlcp = ExtensionUtil.extendLangCompilationProvider(Language.AQL, aqlcp, le);
                         sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
+                        fm = ExtensionUtil.extendFunctionManager(fm, le);
                         break;
                     default:
                         break;
@@ -93,6 +98,8 @@
         this.statementExecutorExtension = see;
         this.aqlCompilationProvider = aqlcp == null ? new AqlCompilationProvider() : aqlcp.second;
         this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
+        this.functionManager =
+                fm == null ? new FunctionManager(FunctionCollection.createDefaultFunctionCollection()) : fm.second;
     }
 
     /** @deprecated use getStatementExecutorFactory instead */
@@ -117,4 +124,9 @@
             default: throw new IllegalArgumentException(String.valueOf(lang));
         }
     }
+
+    @Override
+    public IFunctionManager getFunctionManager() {
+        return functionManager;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
index 07eed0d..380c5a9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.metadata.api.IMetadataExtension;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
@@ -52,18 +53,39 @@
      */
     public static Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
             Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws RuntimeDataException {
-        if (cp != null && le.getLangCompilationProvider(lang) != null) {
+        ILangCompilationProvider lecp = le.getLangCompilationProvider(lang);
+        if (cp != null && lecp != null) {
             throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), cp.first,
                     lang.toString());
         }
-        return (le.getLangCompilationProvider(lang) != null)
-                ? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
+        return lecp != null ? new Pair<>(le.getId(), lecp) : cp;
+    }
+
+    /**
+     * Validate no extension conflict and return function manager extension
+     *
+     * @param fm
+     *            place holder for function manager extension
+     * @param le
+     *            user defined extension
+     * @return the user defined extension
+     * @throws RuntimeDataException
+     *             if extension conflict was detected
+     */
+    public static Pair<ExtensionId, IFunctionManager> extendFunctionManager(Pair<ExtensionId, IFunctionManager> fm,
+            ILangExtension le) throws RuntimeDataException {
+        IFunctionManager lefm = le.getFunctionManager();
+        if (fm != null && lefm != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), fm.first,
+                    IFunctionManager.class.getSimpleName());
+        }
+        return lefm != null ? new Pair<>(le.getId(), lefm) : fm;
     }
 
     /**
      * Validate no extension conflict and return statement executor extension
      *
-     * @param qte
+     * @param see
      *            place holder for statement executor extension
      * @param extension
      *            user defined extension
@@ -71,10 +93,10 @@
      * @throws RuntimeDataException
      *             if extension conflict was detected
      */
-    public static IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
+    public static IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension see,
             IStatementExecutorExtension extension) throws RuntimeDataException {
-        if (qte != null) {
-            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
+        if (see != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, see.getId(), extension.getId(),
                     IStatementExecutorFactory.class.getSimpleName());
         }
         return extension;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
index 3c54c7c..4bcee04 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
@@ -43,7 +43,8 @@
 
     @Test
     public void test() throws Exception {
-        List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+        List<IFunctionDescriptorFactory> functions =
+                FunctionCollection.createDefaultFunctionCollection().getFunctionDescriptorFactories();
         int testedFunctions = 0;
         for (IFunctionDescriptorFactory func : functions) {
             String className = func.getClass().getName();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
index f5e7402..3b33868 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
@@ -43,7 +43,8 @@
 
     @Test
     public void test() throws Exception {
-        List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+        List<IFunctionDescriptorFactory> functions =
+                FunctionCollection.createDefaultFunctionCollection().getFunctionDescriptorFactories();
         int testedFunctions = 0;
         for (IFunctionDescriptorFactory func : functions) {
             String className = func.getClass().getName();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index e2b1761..3f3a27f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -31,7 +31,6 @@
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -107,8 +106,7 @@
                 ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
                 IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
                         edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, null);
-                return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
-                        NonTaggedDataFormat.INSTANCE);
+                return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory);
             case INTERNAL:
                 DataSourceId id = getId();
                 String dataverseName = id.getDataverseName();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 0b6608c..9051087 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -33,7 +33,6 @@
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -170,7 +169,7 @@
         try {
             ARecordType feedOutputType = (ARecordType) itemType;
             ISerializerDeserializer payloadSerde =
-                    NonTaggedDataFormat.INSTANCE.getSerdeProvider().getSerializerDeserializer(feedOutputType);
+                    metadataProvider.getFormat().getSerdeProvider().getSerializerDeserializer(feedOutputType);
             ArrayList<ISerializerDeserializer> serdes = new ArrayList<>();
             serdes.add(payloadSerde);
             if (metaItemType != null) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 9271f332..ef5c256 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -57,6 +57,7 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -78,6 +79,7 @@
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
@@ -143,6 +145,7 @@
     private final ICcApplicationContext appCtx;
     private final IStorageComponentProvider storageComponentProvider;
     private final StorageProperties storageProperties;
+    private final IFunctionManager functionManager;
     private final Dataverse defaultDataverse;
     private final LockList locks;
     private final Map<String, String> config;
@@ -164,6 +167,7 @@
         this.defaultDataverse = defaultDataverse == null ? MetadataBuiltinEntities.DEFAULT_DATAVERSE : defaultDataverse;
         this.storageComponentProvider = appCtx.getStorageComponentProvider();
         storageProperties = appCtx.getStorageProperties();
+        functionManager = ((IFunctionExtensionManager) appCtx.getExtensionManager()).getFunctionManager();
         locks = new LockList();
         config = new HashMap<>();
     }
@@ -257,6 +261,10 @@
     public boolean isTemporaryDatasetWriteJob() {
         // The transaction only writes temporary datasets.
         return isTemporaryDatasetWriteJob;
+    }
+
+    public IFunctionManager getFunctionManager() {
+        return functionManager;
     }
 
     public IDataFormat getFormat() {
@@ -868,13 +876,13 @@
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
-            JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory, IDataFormat format)
+            JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory)
             throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.OBJECT) {
             throw new AlgebricksException("Can only scan datasets of records.");
         }
 
-        ISerializerDeserializer<?> payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
+        ISerializerDeserializer<?> payloadSerde = getFormat().getSerdeProvider().getSerializerDeserializer(itemType);
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         ExternalScanOperatorDescriptor dataScanner =
@@ -1501,7 +1509,7 @@
             // Generate Output Record format
             ISerializerDeserializer<?>[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
             ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
-            ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
+            ISerializerDeserializerProvider serdeProvider = getFormat().getSerdeProvider();
 
             // The order of the output record: propagated variables (including
             // PK and SK), token, and number of token.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index e4a6ca8..3567e81 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -57,7 +57,6 @@
 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.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorDescriptor;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
@@ -422,13 +421,13 @@
 
             // add the previous record first
             int f = 0;
-            outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
+            outputSerDes[f] = metadataProvider.getFormat().getSerdeProvider().getSerializerDeserializer(itemType);
             f++;
             // add the previous meta second
             if (dataset.hasMetaPart()) {
                 outputSerDes[f] =
-                        FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
-                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
+                        metadataProvider.getFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
+                outputTypeTraits[f] = metadataProvider.getFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
                 f++;
             }
             // add the previous filter third
@@ -443,9 +442,9 @@
                     }
                 }
                 fieldIdx = i;
-                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider()
+                outputTypeTraits[f] = metadataProvider.getFormat().getTypeTraitProvider()
                         .getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
-                outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider()
+                outputSerDes[f] = metadataProvider.getFormat().getSerdeProvider()
                         .getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
                 f++;
             }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 445fbd8..6b6b671 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -231,8 +231,8 @@
                 sourceColumn = recordColumn + 1;
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : sourceType, index.getKeyFieldNames().get(i),
-                    sourceColumn);
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
+                    index.getKeyFieldNames().get(i), sourceColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -270,8 +270,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+                            itemType, filterFieldName, numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 96fdf27..60e63c2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -156,8 +156,8 @@
                 sourceColumn = recordColumn + 1;
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyTypes ? enforcedItemType : sourceType, index.getKeyFieldNames().get(i),
-                    sourceColumn);
+                    metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
+                    index.getKeyFieldNames().get(i), sourceColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -185,8 +185,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, recordColumn);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+                            itemType, filterFieldName, recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index 15f8a23..1c353a8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -29,7 +29,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.om.utils.RecordUtil;
-import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -108,13 +107,13 @@
         ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
         if (numSecondaryKeys > 0) {
-            secondaryFieldAccessEvalFactories[0] = FormatUtils.getDefaultFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : itemType, index.getKeyFieldNames().get(0),
-                    recordColumn);
+            secondaryFieldAccessEvalFactories[0] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                    index.getKeyFieldNames().get(0), recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -124,8 +123,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = FormatUtils.getDefaultFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, recordColumn);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+                            itemType, filterFieldName, recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 8fd8a7a..6040eaa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -89,9 +89,9 @@
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
-        secondaryFieldAccessEvalFactories =
-                metadataProvider.getFormat().createMBRFactory(isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                        secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+        secondaryFieldAccessEvalFactories = metadataProvider.getFormat().createMBRFactory(
+                metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 171d72e..b99ae2f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -32,7 +32,6 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexBulkLoadOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
@@ -155,10 +154,6 @@
     @Override
     protected AlgebricksMetaOperatorDescriptor createCastOp(JobSpecification spec, DatasetType dsType,
             boolean strictCast) throws AlgebricksException {
-        IFunctionDescriptor castFuncDesc = FunctionManagerHolder.getFunctionManager()
-                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
-        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
-
         int[] outColumns = new int[1];
 
         // tags(2) + primary keys + record + meta part(?)
@@ -185,7 +180,7 @@
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = castFuncDesc.createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { getTaggedRecordDescriptor(enforcedRecDesc) });
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 8fc9ed7..98498ac 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -48,7 +48,6 @@
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.IsUnknownDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -294,10 +293,6 @@
 
     protected AlgebricksMetaOperatorDescriptor createCastOp(JobSpecification spec, DatasetType dsType,
             boolean strictCast) throws AlgebricksException {
-        IFunctionDescriptor castFuncDesc = FunctionManagerHolder.getFunctionManager()
-                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
-        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
-
         int[] outColumns = new int[1];
         int[] projectionList = new int[(dataset.hasMetaPart() ? 2 : 1) + numPrimaryKeys];
         int recordIdx;
@@ -318,12 +313,19 @@
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = castFuncDesc.createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { enforcedRecDesc });
     }
 
+    protected IFunctionDescriptor createCastFunction(boolean strictCast) throws AlgebricksException {
+        IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
+                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
+        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
+        return castFuncDesc;
+    }
+
     protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
             IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
         int[] sortFields = new int[secondaryComparatorFactories.length];
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index b4d8a22..b230aec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -30,7 +30,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.om.utils.RecordUtil;
-import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -111,12 +110,12 @@
         ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
         if (numSecondaryKeys > 0) {
-            secondaryFieldAccessEvalFactories[0] = FormatUtils.getDefaultFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : itemType, index.getKeyFieldNames().get(0),
-                    numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[0] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                    index.getKeyFieldNames().get(0), numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -126,8 +125,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = FormatUtils.getDefaultFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+                            itemType, filterFieldName, numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 08ee0aa..ca10c28 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -99,9 +99,9 @@
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
-        secondaryFieldAccessEvalFactories =
-                metadataProvider.getFormat().createMBRFactory(isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                        secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+        secondaryFieldAccessEvalFactories = metadataProvider.getFormat().createMBRFactory(
+                metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
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 7624a8a..678a864 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
@@ -20,16 +20,13 @@
 
 import java.util.List;
 
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -69,18 +66,16 @@
     public IMissingWriterFactory getMissingWriterFactory();
 
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            ARecordType recType, List<String> fldName) throws AlgebricksException;
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException;
 
-    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName,
-            int recordColumn) throws AlgebricksException;
-
-    public IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment typeEnvironment)
-            throws AlgebricksException;
+    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn) throws AlgebricksException;
 
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException;
 
-    public IScalarEvaluatorFactory[] createMBRFactory(ARecordType recType, List<String> fldName, int recordColumn,
-            int dimension, List<String> filterFieldName, boolean isPointMBR) throws AlgebricksException;
+    public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
+            throws AlgebricksException;
 
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
index 526eab6..0e746f9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
@@ -26,5 +26,13 @@
      *
      * @return a new IFunctionDescriptor instance
      */
-    public IFunctionDescriptor createFunctionDescriptor();
+    IFunctionDescriptor createFunctionDescriptor();
+
+    /**
+     * Creates type inferer for this function
+     * @return an new instance of the type inferer or {@code null} if not provided
+     */
+    default IFunctionTypeInferer createFunctionTypeInferer() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
new file mode 100644
index 0000000..6f80c86
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.functions;
+
+public interface IFunctionExtensionManager {
+    /**
+     * Returns function manager
+     *
+     * @return a new {@link IFunctionManager} instance
+     */
+    IFunctionManager getFunctionManager();
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
index f7c11ff..4fce41d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
@@ -21,11 +21,9 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
-public interface IFunctionManager extends Iterable<IFunctionDescriptorFactory> {
+public interface IFunctionManager {
 
-    public void registerFunction(IFunctionDescriptorFactory descriptorFactory);
+    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
 
-    public void unregisterFunction(IFunctionDescriptorFactory descriptorFactory);
-
-    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
new file mode 100644
index 0000000..b8e27e1
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.functions;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public interface IFunctionTypeInferer {
+    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+            throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index f117d14..4e35950 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -21,22 +21,29 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class ListifyAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ListifyAggregateDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AOrderedListType oltype;
 
     @Override
@@ -53,5 +60,4 @@
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new ListifyAggregateFunctionEvalFactory(args, oltype);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index 96a298f..643098f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.runtime.aggregates.scalar;
 
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor.ScanCollectionUnnestingFunctionFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -37,6 +33,12 @@
 public abstract class AbstractScalarAggregateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
+    private final AbstractAggregateFunctionDynamicDescriptor aggFuncDesc;
+
+    protected AbstractScalarAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        this.aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) aggFuncDesc;
+    }
+
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
             throws AlgebricksException {
@@ -45,10 +47,6 @@
         IScalarEvaluatorFactory[] aggFuncArgs = new IScalarEvaluatorFactory[1];
         aggFuncArgs[0] = new ColumnAccessEvalFactory(0);
         // Create aggregate function from this scalar version.
-        FunctionIdentifier fid = BuiltinFunctions.getAggregateFunction(getIdentifier());
-        IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
-        IFunctionDescriptor fd = mgr.lookupFunction(fid);
-        AbstractAggregateFunctionDynamicDescriptor aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) fd;
         final IAggregateEvaluatorFactory aggFuncFactory = aggFuncDesc.createAggregateEvaluatorFactory(aggFuncArgs);
 
         return new IScalarEvaluatorFactory() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
index 51419f1..9fb994a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarAvgAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_AVG;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarAvgAggregateDescriptor();
+            return new ScalarAvgAggregateDescriptor(AvgAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarAvgAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
index 5bdb8b1..6aba99f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarCountAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_COUNT;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarCountAggregateDescriptor();
+            return new ScalarCountAggregateDescriptor(CountAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarCountAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
index f411bea..f18fd32 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarMaxAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_MAX;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarMaxAggregateDescriptor();
+            return new ScalarMaxAggregateDescriptor(MaxAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarMaxAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
index b1f95b0..17d008c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.MinAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarMinAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_MIN;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarMinAggregateDescriptor();
+            return new ScalarMinAggregateDescriptor(MinAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarMinAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
index 82c301c..ecc10d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlAvgAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlAvgAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_AVG;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlAvgAggregateDescriptor();
+            return new ScalarSqlAvgAggregateDescriptor(SqlAvgAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlAvgAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
index baea906..029956a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
@@ -21,18 +21,27 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlCountAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_COUNT;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlCountAggregateDescriptor();
+            return new ScalarSqlCountAggregateDescriptor(
+                    SqlCountAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlCountAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
index a182777..fb616b4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlMaxAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_MAX;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlMaxAggregateDescriptor();
+            return new ScalarSqlMaxAggregateDescriptor(SqlMaxAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    public ScalarSqlMaxAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
index 85da97c..123b955 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlMinAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlMinAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_MIN;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlMinAggregateDescriptor();
+            return new ScalarSqlMinAggregateDescriptor(SqlMinAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlMinAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
index fae1a1a..078ea57 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlSumAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_SUM;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlSumAggregateDescriptor();
+            return new ScalarSqlSumAggregateDescriptor(SqlSumAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
index d534193..4d547e1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSumAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SUM;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSumAggregateDescriptor();
+            return new ScalarSumAggregateDescriptor(SumAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
index 5cac892..f75f975 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
@@ -21,22 +21,29 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class ClosedRecordConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ClosedRecordConstructorDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
         }
     };
 
     private static final long serialVersionUID = 1L;
-
     private ARecordType recType;
 
     @Override
@@ -53,5 +60,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new ClosedRecordConstructorEvalFactory(args, recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
index 0b26583..7348fd8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
@@ -25,8 +25,10 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -44,6 +46,10 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OpenRecordConstructorDescriptor();
         }
+
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.OpenRecordConstructorTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
index 977101a..3a378de 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
@@ -25,9 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -40,14 +42,19 @@
 
 public class OrderedListConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OrderedListConstructorDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AOrderedListType oltype;
 
     @Override
@@ -69,9 +76,7 @@
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory[] args;
-
         private boolean selfDescList = false;
-
         private AOrderedListType orderedlistType;
 
         public OrderedListConstructorEvaluatorFactory(IScalarEvaluatorFactory[] args, AOrderedListType type) {
@@ -142,6 +147,5 @@
             };
 
         }
-
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
index 50b7699..5f11acb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
@@ -25,10 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -41,14 +42,19 @@
 
 public class UnorderedListConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new UnorderedListConstructorDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AUnorderedListType ultype;
 
     @Override
@@ -70,7 +76,6 @@
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory[] args;
-
         private boolean selfDescList = false;
         private boolean homoList = false;
         private AUnorderedListType unorderedlistType;
@@ -144,5 +149,4 @@
 
         }
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
index 7f5c58d..efc5f8a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
@@ -21,9 +21,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -37,10 +39,14 @@
 public class CastTypeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CastTypeDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.CastTypeInferer();
         }
     };
 
@@ -84,4 +90,3 @@
         };
     }
 }
-
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
index eea1b6e..51ea579 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
@@ -22,9 +22,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -50,6 +52,11 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CastTypeLaxDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.CastTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
index 150c28e..51497ed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
@@ -26,12 +26,14 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,10 +45,16 @@
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class DeepEqualityDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new DeepEqualityDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.DeepEqualityTypeInferer();
         }
     };
 
@@ -59,7 +67,6 @@
         this.inputTypeLeft = (IAType) states[0];
         this.inputTypeRight = (IAType) states[1];
     }
-
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
index 7ce7dec..d641ea9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class FieldAccessByIndexDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessByIndexDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FieldAccessByIndexTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new FieldAccessByIndexEvalFactory(args[0], args[1], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
index 95b06f7..5dd3e19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
@@ -23,20 +23,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class FieldAccessNestedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessNestedDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FieldAccessNestedTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
     private List<String> fldName;
 
@@ -55,5 +63,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new FieldAccessNestedEvalFactory(args[0], recType, fldName);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
index 3419b68..631dd70 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class GetRecordFieldValueDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GetRecordFieldValueDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.GetRecordFieldValueTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new GetRecordFieldValueEvalFactory(args[0], args[1], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
index 6205d77..273e3ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class GetRecordFieldsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GetRecordFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.GetRecordFieldsTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new GetRecordFieldsEvalFactory(args[0], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
index 5df04f8..97a9ee0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
@@ -31,6 +31,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.PointableAllocator;
@@ -47,6 +48,7 @@
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -67,7 +69,13 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordAddFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordAddFieldsTypeInferer();
+        }
     };
+
     private static final long serialVersionUID = 1L;
     private ARecordType outRecType;
     private ARecordType inRecType;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index 0b432ac..ce7ed6e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -42,6 +43,7 @@
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -68,7 +70,13 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordMergeDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordMergeTypeInferer();
+        }
     };
+
     private static final long serialVersionUID = 1L;
     private ARecordType outRecType;
     private ARecordType inRecType0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index 7158d2d..e5ed628 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -37,6 +38,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -54,6 +56,11 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordPairsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordPairsTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
index 092fbfb..909c74b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
@@ -21,28 +21,35 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class RecordRemoveFieldsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordRemoveFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordRemoveFieldsTypeInferer();
+        }
     };
 
-    private RecordRemoveFieldsDescriptor() {
-    }
-
+    private static final long serialVersionUID = 1L;
     private ARecordType outputRecordType;
     private ARecordType inputRecType;
     private AOrderedListType inputListType;
+
+    private RecordRemoveFieldsDescriptor() {
+    }
 
     @Override
     public void setImmutableStates(Object... states) {
@@ -60,5 +67,4 @@
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.REMOVE_FIELDS;
     }
-
 }
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 090352d..4890b34 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
@@ -19,11 +19,8 @@
 package org.apache.asterix.runtime.formats;
 
 import java.io.DataOutput;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -51,32 +48,19 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionManager;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
-import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
-import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableEvalSizeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -94,35 +78,16 @@
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 
 public class NonTaggedDataFormat implements IDataFormat {
 
-    public static final NonTaggedDataFormat INSTANCE = new NonTaggedDataFormat();
+    static final NonTaggedDataFormat INSTANCE = new NonTaggedDataFormat();
 
     private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-    private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<>();
-
     public static final String NON_TAGGED_DATA_FORMAT = "org.apache.asterix.runtime.formats.NonTaggedDataFormat";
 
-    private final Map<FunctionIdentifier, FunctionTypeInferer> functionTypeInferers = new HashMap<>();
-
-    static {
-        typeToValueParserFactMap.put(ATypeTag.INTEGER, IntegerParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.BIGINT, LongParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-    }
-
-    public NonTaggedDataFormat() {
-        registerTypeInferers();
+    private NonTaggedDataFormat() {
     }
 
     @Override
@@ -152,8 +117,8 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName,
-            int recordColumn) throws AlgebricksException {
+    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn) throws AlgebricksException {
         IScalarEvaluatorFactory recordEvalFactory = new ColumnAccessEvalFactory(recordColumn);
 
         if (fldName.size() == 1) {
@@ -173,8 +138,7 @@
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                            .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
                     fDesc.setImmutableStates(recType);
                     return fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -190,16 +154,14 @@
                 }
                 IScalarEvaluatorFactory fldNameEvalFactory =
                         new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                        .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
                 return fDesc.createEvaluatorFactory(
                         new IScalarEvaluatorFactory[] { recordEvalFactory, fldNameEvalFactory });
             }
         }
 
         if (fldName.size() > 1) {
-            IFunctionDescriptor fDesc =
-                    FunctionManagerHolder.getFunctionManager().lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
             fDesc.setImmutableStates(recType, fldName);
             return fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
         }
@@ -209,9 +171,11 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public IScalarEvaluatorFactory[] createMBRFactory(ARecordType recType, List<String> fldName, int recordColumn,
-            int dimension, List<String> filterFieldName, boolean isPointMBR) throws AlgebricksException {
-        IScalarEvaluatorFactory evalFactory = getFieldAccessEvaluatorFactory(recType, fldName, recordColumn);
+    public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
+            throws AlgebricksException {
+        IScalarEvaluatorFactory evalFactory =
+                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn);
         int numOfFields = isPointMBR ? dimension : dimension * 2;
         IScalarEvaluatorFactory[] evalFactories =
                 new IScalarEvaluatorFactory[numOfFields + (filterFieldName == null ? 0 : 1)];
@@ -242,7 +206,8 @@
             evalFactories[i] = new CreateMBREvalFactory(evalFactory, dimensionEvalFactory, coordinateEvalFactory);
         }
         if (filterFieldName != null) {
-            evalFactories[numOfFields] = getFieldAccessEvaluatorFactory(recType, filterFieldName, recordColumn);
+            evalFactories[numOfFields] =
+                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn);
         }
         return evalFactories;
     }
@@ -250,7 +215,7 @@
     @SuppressWarnings("unchecked")
     @Override
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            ARecordType recType, List<String> fldName) throws AlgebricksException {
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException {
         String[] names = recType.getFieldNames();
         int n = names.length;
         if (fldName.size() > 1) {
@@ -269,8 +234,7 @@
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                            .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
                     fDesc.setImmutableStates(recType);
                     IScalarEvaluatorFactory evalFactory = fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -278,11 +242,9 @@
                             BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
 
                     ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                            new MutableObject<ILogicalExpression>(
-                                    new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
-                    return new Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory,
-                            partitionFun, recType.getFieldTypes()[i]);
+                            new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                            new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    return new Triple<>(evalFactory, partitionFun, recType.getFieldTypes()[i]);
                 }
             }
         } else {
@@ -296,286 +258,18 @@
             } catch (HyracksDataException e) {
                 throw new AlgebricksException(e);
             }
-            IFunctionDescriptor fDesc =
-                    FunctionManagerHolder.getFunctionManager().lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
             fDesc.setImmutableStates(recType, fldName);
             IScalarEvaluatorFactory evalFactory =
                     fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
             IFunctionInfo finfoAccess = BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED);
 
             ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(as))));
-            return new Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory, partitionFun,
-                    recType.getSubFieldType(fldName));
+                    new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                    new MutableObject<>(new ConstantExpression(new AsterixConstantValue(as))));
+            return new Triple<>(evalFactory, partitionFun, recType.getSubFieldType(fldName));
         }
         throw new AlgebricksException("Could not find field " + fldName + " in the schema.");
-    }
-
-    @Override
-    public IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment context)
-            throws AlgebricksException {
-        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
-        IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
-        IFunctionDescriptor fd = mgr.lookupFunction(fnId);
-        if (fd == null) {
-            throw new AlgebricksException("Unresolved function " + fnId);
-        }
-        final FunctionIdentifier fid = fd.getIdentifier();
-        if (functionTypeInferers.containsKey(fid)) {
-            functionTypeInferers.get(fid).infer(expr, fd, context);
-        }
-        return fd;
-    }
-
-    interface FunctionTypeInferer {
-        void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException;
-    }
-
-    void registerTypeInferers() {
-        functionTypeInferers.put(BuiltinFunctions.LISTIFY, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.RECORD_MERGE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.DEEP_EQUAL, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
-                fd.setImmutableStates(type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.ADD_FIELDS, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                ILogicalExpression listExpr = f.getArguments().get(1).getValue();
-                IAType type1 = (IAType) context.getType(listExpr);
-                if (type0.getTypeTag().equals(ATypeTag.ANY)) {
-                    type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                }
-                if (type1.getTypeTag().equals(ATypeTag.ANY)) {
-                    type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                }
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.REMOVE_FIELDS, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                ILogicalExpression le = f.getArguments().get(1).getValue();
-                IAType type1 = (IAType) context.getType(le);
-                if (type0.getTypeTag().equals(ATypeTag.ANY)) {
-                    type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                }
-                if (type1.getTypeTag().equals(ATypeTag.ANY)) {
-                    type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                }
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CAST_TYPE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                IAType rt = TypeCastUtils.getRequiredType(funcExpr);
-                IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
-                fd.setImmutableStates(rt, it);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CAST_TYPE_LAX, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                IAType rt = TypeCastUtils.getRequiredType(funcExpr);
-                IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
-                fd.setImmutableStates(rt, it);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                ARecordType rt = (ARecordType) context.getType(expr);
-                fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
-            }
-
-            private boolean[] computeOpenFields(AbstractFunctionCallExpression expr, ARecordType recType) {
-                int n = expr.getArguments().size() / 2;
-                boolean[] open = new boolean[n];
-                for (int i = 0; i < n; i++) {
-                    Mutable<ILogicalExpression> argRef = expr.getArguments().get(2 * i);
-                    ILogicalExpression arg = argRef.getValue();
-                    open[i] = true;
-                    final String fn = ConstantExpressionUtil.getStringConstant(arg);
-                    if (fn != null) {
-                        for (String s : recType.getFieldNames()) {
-                            if (s.equals(fn)) {
-                                open[i] = false;
-                                break;
-                            }
-                        }
-                    }
-                }
-                return open;
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                switch (t.getTypeTag()) {
-                    case OBJECT: {
-                        fd.setImmutableStates(t);
-                        break;
-                    }
-                    case UNION: {
-                        AUnionType unionT = (AUnionType) t;
-                        if (unionT.isUnknownableType()) {
-                            IAType t2 = unionT.getActualType();
-                            if (t2.getTypeTag() == ATypeTag.OBJECT) {
-                                fd.setImmutableStates(t2);
-                                break;
-                            }
-                        }
-                        throw new NotImplementedException("field-access-by-index for data of type " + t);
-                    }
-                    default: {
-                        throw new NotImplementedException("field-access-by-index for data of type " + t);
-                    }
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.FIELD_ACCESS_NESTED, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                AOrderedList fieldPath = (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce
-                        .getArguments().get(1).getValue()).getValue()).getObject());
-                List<String> listFieldPath = new ArrayList<String>();
-                for (int i = 0; i < fieldPath.size(); i++) {
-                    listFieldPath.add(((AString) fieldPath.getItem(i)).getStringValue());
-                }
-
-                switch (t.getTypeTag()) {
-                    case OBJECT: {
-                        fd.setImmutableStates(t, listFieldPath);
-                        break;
-                    }
-                    case ANY:
-                        fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
-                        break;
-                    default: {
-                        throw new NotImplementedException("field-access-nested for data of type " + t);
-                    }
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.GET_RECORD_FIELDS, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("get-record-fields for data of type " + t);
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.GET_RECORD_FIELD_VALUE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("get-record-field-value for data of type " + t);
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.RECORD_PAIRS, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("record-fields with data of type " + t);
-                }
-            }
-        });
     }
 
     @Override
@@ -601,7 +295,7 @@
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException {
-        IAObject obj = null;
+        IAObject obj;
         if (value.isMissing()) {
             obj = AMissing.MISSING;
         } else if (value.isTrue()) {
@@ -634,58 +328,54 @@
 
     @Override
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
-        return new IExpressionEvalSizeComputer() {
-            @Override
-            public int getEvalSize(ILogicalExpression expr, IVariableEvalSizeEnvironment env)
-                    throws AlgebricksException {
-                switch (expr.getExpressionTag()) {
-                    case CONSTANT: {
-                        ConstantExpression c = (ConstantExpression) expr;
-                        if (c == ConstantExpression.MISSING) {
-                            return 1;
-                        } else if (c == ConstantExpression.FALSE || c == ConstantExpression.TRUE) {
-                            return 2;
-                        } else {
-                            AsterixConstantValue acv = (AsterixConstantValue) c.getValue();
-                            IAObject o = acv.getObject();
-                            switch (o.getType().getTypeTag()) {
-                                case DOUBLE:
-                                    return 9;
-                                case FLOAT:
-                                    return 5;
-                                case BOOLEAN:
-                                    return 2;
-                                case MISSING:
-                                    return 1;
-                                case NULL:
-                                    return 1;
-                                case TINYINT:
-                                    return 2;
-                                case SMALLINT:
-                                    return 3;
-                                case INTEGER:
-                                    return 5;
-                                case BIGINT:
-                                    return 9;
-                                default:
-                                    return -1;
-                            }
+        return (expr, env) -> {
+            switch (expr.getExpressionTag()) {
+                case CONSTANT: {
+                    ConstantExpression c = (ConstantExpression) expr;
+                    if (c == ConstantExpression.MISSING) {
+                        return 1;
+                    } else if (c == ConstantExpression.FALSE || c == ConstantExpression.TRUE) {
+                        return 2;
+                    } else {
+                        AsterixConstantValue acv = (AsterixConstantValue) c.getValue();
+                        IAObject o = acv.getObject();
+                        switch (o.getType().getTypeTag()) {
+                            case DOUBLE:
+                                return 9;
+                            case FLOAT:
+                                return 5;
+                            case BOOLEAN:
+                                return 2;
+                            case MISSING:
+                                return 1;
+                            case NULL:
+                                return 1;
+                            case TINYINT:
+                                return 2;
+                            case SMALLINT:
+                                return 3;
+                            case INTEGER:
+                                return 5;
+                            case BIGINT:
+                                return 9;
+                            default:
+                                return -1;
                         }
                     }
-                    case FUNCTION_CALL: {
-                        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                        if (f.getFunctionIdentifier().equals(BuiltinFunctions.TID)) {
-                            return 5;
-                        } else {
-                            // TODO
-                            return -1;
-                        }
-                    }
-                    default: {
+                }
+                case FUNCTION_CALL: {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+                    if (f.getFunctionIdentifier().equals(BuiltinFunctions.TID)) {
+                        return 5;
+                    } else {
                         // TODO
                         return -1;
                     }
                 }
+                default: {
+                    // TODO
+                    return -1;
+                }
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index c02732f..5acbeb9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -331,394 +331,400 @@
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
 
 /**
- * This class (statically) holds a list of function descriptor factories.
+ * This class holds a list of function descriptor factories.
  */
 public final class FunctionCollection {
 
     private static final String FACTORY = "FACTORY";
-    private static final List<IFunctionDescriptorFactory> temp = new ArrayList<>();
 
-    static {
+    private final List<IFunctionDescriptorFactory> descriptorFactories = new ArrayList<>();
+
+    public void add(IFunctionDescriptorFactory descriptorFactory) {
+        descriptorFactories.add(descriptorFactory);
+    }
+
+    public void addGenerated(IFunctionDescriptorFactory descriptorFactory) {
+        add(getGeneratedFunctionDescriptorFactory(descriptorFactory.createFunctionDescriptor().getClass()));
+    }
+
+    public static FunctionCollection createDefaultFunctionCollection() {
+        FunctionCollection fc = new FunctionCollection();
+
         // unnesting function
-        temp.add(TidRunningAggregateDescriptor.FACTORY);
-        temp.add(ScanCollectionDescriptor.FACTORY);
-        temp.add(RangeDescriptor.FACTORY);
-        temp.add(SubsetCollectionDescriptor.FACTORY);
+        fc.add(TidRunningAggregateDescriptor.FACTORY);
+        fc.add(ScanCollectionDescriptor.FACTORY);
+        fc.add(RangeDescriptor.FACTORY);
+        fc.add(SubsetCollectionDescriptor.FACTORY);
 
         // aggregate functions
-        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);
-        temp.add(FirstElementAggregateDescriptor.FACTORY);
-        temp.add(LocalFirstElementAggregateDescriptor.FACTORY);
+        fc.add(ListifyAggregateDescriptor.FACTORY);
+        fc.add(CountAggregateDescriptor.FACTORY);
+        fc.add(AvgAggregateDescriptor.FACTORY);
+        fc.add(LocalAvgAggregateDescriptor.FACTORY);
+        fc.add(IntermediateAvgAggregateDescriptor.FACTORY);
+        fc.add(GlobalAvgAggregateDescriptor.FACTORY);
+        fc.add(SumAggregateDescriptor.FACTORY);
+        fc.add(LocalSumAggregateDescriptor.FACTORY);
+        fc.add(MaxAggregateDescriptor.FACTORY);
+        fc.add(LocalMaxAggregateDescriptor.FACTORY);
+        fc.add(MinAggregateDescriptor.FACTORY);
+        fc.add(LocalMinAggregateDescriptor.FACTORY);
+        fc.add(FirstElementAggregateDescriptor.FACTORY);
+        fc.add(LocalFirstElementAggregateDescriptor.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);
+        fc.add(SerializableCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableLocalAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableSumAggregateDescriptor.FACTORY);
+        fc.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);
-        temp.add(EmptyStreamAggregateDescriptor.FACTORY);
-        temp.add(NonEmptyStreamAggregateDescriptor.FACTORY);
+        fc.add(ScalarCountAggregateDescriptor.FACTORY);
+        fc.add(ScalarAvgAggregateDescriptor.FACTORY);
+        fc.add(ScalarSumAggregateDescriptor.FACTORY);
+        fc.add(ScalarMaxAggregateDescriptor.FACTORY);
+        fc.add(ScalarMinAggregateDescriptor.FACTORY);
+        fc.add(EmptyStreamAggregateDescriptor.FACTORY);
+        fc.add(NonEmptyStreamAggregateDescriptor.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);
+        fc.add(SqlCountAggregateDescriptor.FACTORY);
+        fc.add(SqlAvgAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(IntermediateSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(GlobalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SqlSumAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlSumAggregateDescriptor.FACTORY);
+        fc.add(SqlMaxAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlMaxAggregateDescriptor.FACTORY);
+        fc.add(SqlMinAggregateDescriptor.FACTORY);
+        fc.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);
+        fc.add(SerializableSqlCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableLocalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableSqlSumAggregateDescriptor.FACTORY);
+        fc.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);
+        fc.add(ScalarSqlCountAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlSumAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlMaxAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlMinAggregateDescriptor.FACTORY);
 
         // boolean functions
-        temp.add(AndDescriptor.FACTORY);
-        temp.add(OrDescriptor.FACTORY);
+        fc.add(AndDescriptor.FACTORY);
+        fc.add(OrDescriptor.FACTORY);
 
         // Record constructors
-        temp.add(ClosedRecordConstructorDescriptor.FACTORY);
-        temp.add(OpenRecordConstructorDescriptor.FACTORY);
+        fc.add(ClosedRecordConstructorDescriptor.FACTORY);
+        fc.add(OpenRecordConstructorDescriptor.FACTORY);
 
         // List constructors
-        temp.add(OrderedListConstructorDescriptor.FACTORY);
-        temp.add(UnorderedListConstructorDescriptor.FACTORY);
+        fc.add(OrderedListConstructorDescriptor.FACTORY);
+        fc.add(UnorderedListConstructorDescriptor.FACTORY);
 
         // Sleep function
-        temp.add(SleepDescriptor.FACTORY);
+        fc.add(SleepDescriptor.FACTORY);
 
         // Inject failure function
-        temp.add(InjectFailureDescriptor.FACTORY);
+        fc.add(InjectFailureDescriptor.FACTORY);
 
         // Switch case
-        temp.add(SwitchCaseDescriptor.FACTORY);
+        fc.add(SwitchCaseDescriptor.FACTORY);
 
         // null functions
-        temp.add(IsMissingDescriptor.FACTORY);
-        temp.add(IsNullDescriptor.FACTORY);
-        temp.add(IsUnknownDescriptor.FACTORY);
-        temp.add(IsSystemNullDescriptor.FACTORY);
-        temp.add(CheckUnknownDescriptor.FACTORY);
-        temp.add(IfMissingDescriptor.FACTORY);
-        temp.add(IfNullDescriptor.FACTORY);
-        temp.add(IfMissingOrNullDescriptor.FACTORY);
+        fc.add(IsMissingDescriptor.FACTORY);
+        fc.add(IsNullDescriptor.FACTORY);
+        fc.add(IsUnknownDescriptor.FACTORY);
+        fc.add(IsSystemNullDescriptor.FACTORY);
+        fc.add(CheckUnknownDescriptor.FACTORY);
+        fc.add(IfMissingDescriptor.FACTORY);
+        fc.add(IfNullDescriptor.FACTORY);
+        fc.add(IfMissingOrNullDescriptor.FACTORY);
 
         // uuid generators (zero independent functions)
-        temp.add(CreateUUIDDescriptor.FACTORY);
-        temp.add(UUIDDescriptor.FACTORY);
-        temp.add(CreateQueryUIDDescriptor.FACTORY);
-        temp.add(CurrentDateDescriptor.FACTORY);
-        temp.add(CurrentTimeDescriptor.FACTORY);
-        temp.add(CurrentDateTimeDescriptor.FACTORY);
+        fc.add(CreateUUIDDescriptor.FACTORY);
+        fc.add(UUIDDescriptor.FACTORY);
+        fc.add(CreateQueryUIDDescriptor.FACTORY);
+        fc.add(CurrentDateDescriptor.FACTORY);
+        fc.add(CurrentTimeDescriptor.FACTORY);
+        fc.add(CurrentDateTimeDescriptor.FACTORY);
 
         // TODO: decide how should we deal these two weird functions as
         // the number of arguments of the function depend on the first few arguments.
-        temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
-        temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
+        fc.add(SimilarityJaccardPrefixDescriptor.FACTORY);
+        fc.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
 
         // functions that need generated class for null-handling.
-        List<IFunctionDescriptorFactory> functionsToInjectUnkownHandling = new ArrayList<>();
 
         // Element accessors.
-        functionsToInjectUnkownHandling.add(FieldAccessByIndexDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FieldAccessByNameDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FieldAccessNestedDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AnyCollectionMemberDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetItemDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessByIndexDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessByNameDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessNestedDescriptor.FACTORY);
+
+        fc.addGenerated(AnyCollectionMemberDescriptor.FACTORY);
+        fc.addGenerated(GetItemDescriptor.FACTORY);
 
         // Numeric functions
-        functionsToInjectUnkownHandling.add(NumericUnaryMinusDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericAddDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericDivideDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericMultiplyDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSubDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericModuloDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCaretDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NotDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LenDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericAbsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCeilingDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericFloorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundHalfToEvenDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundHalfToEven2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericACosDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericASinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericATanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCosDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericTanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericExpDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericLnDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericLogDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSqrtDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSignDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericTruncDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericATan2Descriptor.FACTORY);
+        fc.addGenerated(NumericUnaryMinusDescriptor.FACTORY);
+        fc.addGenerated(NumericAddDescriptor.FACTORY);
+        fc.addGenerated(NumericDivideDescriptor.FACTORY);
+        fc.addGenerated(NumericMultiplyDescriptor.FACTORY);
+        fc.addGenerated(NumericSubDescriptor.FACTORY);
+        fc.addGenerated(NumericModuloDescriptor.FACTORY);
+        fc.addGenerated(NumericCaretDescriptor.FACTORY);
+        fc.addGenerated(NotDescriptor.FACTORY);
+        fc.addGenerated(LenDescriptor.FACTORY);
+        fc.addGenerated(NumericAbsDescriptor.FACTORY);
+        fc.addGenerated(NumericCeilingDescriptor.FACTORY);
+        fc.addGenerated(NumericFloorDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundHalfToEvenDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundHalfToEven2Descriptor.FACTORY);
+        fc.addGenerated(NumericACosDescriptor.FACTORY);
+        fc.addGenerated(NumericASinDescriptor.FACTORY);
+        fc.addGenerated(NumericATanDescriptor.FACTORY);
+        fc.addGenerated(NumericCosDescriptor.FACTORY);
+        fc.addGenerated(NumericSinDescriptor.FACTORY);
+        fc.addGenerated(NumericTanDescriptor.FACTORY);
+        fc.addGenerated(NumericExpDescriptor.FACTORY);
+        fc.addGenerated(NumericLnDescriptor.FACTORY);
+        fc.addGenerated(NumericLogDescriptor.FACTORY);
+        fc.addGenerated(NumericSqrtDescriptor.FACTORY);
+        fc.addGenerated(NumericSignDescriptor.FACTORY);
+        fc.addGenerated(NumericTruncDescriptor.FACTORY);
+        fc.addGenerated(NumericATan2Descriptor.FACTORY);
 
         // Comparisons.
-        functionsToInjectUnkownHandling.add(EqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GreaterThanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GreaterThanOrEqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LessThanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LessThanOrEqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NotEqualsDescriptor.FACTORY);
+        fc.addGenerated(EqualsDescriptor.FACTORY);
+        fc.addGenerated(GreaterThanDescriptor.FACTORY);
+        fc.addGenerated(GreaterThanOrEqualsDescriptor.FACTORY);
+        fc.addGenerated(LessThanDescriptor.FACTORY);
+        fc.addGenerated(LessThanOrEqualsDescriptor.FACTORY);
+        fc.addGenerated(NotEqualsDescriptor.FACTORY);
 
         // Binary functions
-        functionsToInjectUnkownHandling.add(BinaryLengthDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(BinaryConcatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubBinaryFromDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubBinaryFromToDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FindBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FindBinaryFromDescriptor.FACTORY);
+        fc.addGenerated(BinaryLengthDescriptor.FACTORY);
+        fc.addGenerated(ParseBinaryDescriptor.FACTORY);
+        fc.addGenerated(PrintBinaryDescriptor.FACTORY);
+        fc.addGenerated(BinaryConcatDescriptor.FACTORY);
+        fc.addGenerated(SubBinaryFromDescriptor.FACTORY);
+        fc.addGenerated(SubBinaryFromToDescriptor.FACTORY);
+        fc.addGenerated(FindBinaryDescriptor.FACTORY);
+        fc.addGenerated(FindBinaryFromDescriptor.FACTORY);
 
         // String functions
-        functionsToInjectUnkownHandling.add(StringLikeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringEndsWithDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringStartsWithDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringEqualDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLowerCaseDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringUpperCaseDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLengthDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(Substring2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringBeforeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringAfterDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringToCodePointDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CodePointToStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringConcatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringJoinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpContainsWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpLikeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpLikeWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpPositionDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpPositionWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpReplaceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpReplaceWithFlagsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringInitCapDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringPositionDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRepeatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringSplitDescriptor.FACTORY);
+        fc.addGenerated(StringLikeDescriptor.FACTORY);
+        fc.addGenerated(StringContainsDescriptor.FACTORY);
+        fc.addGenerated(StringEndsWithDescriptor.FACTORY);
+        fc.addGenerated(StringStartsWithDescriptor.FACTORY);
+        fc.addGenerated(SubstringDescriptor.FACTORY);
+        fc.addGenerated(StringEqualDescriptor.FACTORY);
+        fc.addGenerated(StringLowerCaseDescriptor.FACTORY);
+        fc.addGenerated(StringUpperCaseDescriptor.FACTORY);
+        fc.addGenerated(StringLengthDescriptor.FACTORY);
+        fc.addGenerated(Substring2Descriptor.FACTORY);
+        fc.addGenerated(SubstringBeforeDescriptor.FACTORY);
+        fc.addGenerated(SubstringAfterDescriptor.FACTORY);
+        fc.addGenerated(StringToCodePointDescriptor.FACTORY);
+        fc.addGenerated(CodePointToStringDescriptor.FACTORY);
+        fc.addGenerated(StringConcatDescriptor.FACTORY);
+        fc.addGenerated(StringJoinDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpContainsDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpContainsWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpLikeDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpLikeWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpPositionDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpPositionWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpReplaceDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpReplaceWithFlagsDescriptor.FACTORY);
+        fc.addGenerated(StringInitCapDescriptor.FACTORY);
+        fc.addGenerated(StringTrimDescriptor.FACTORY);
+        fc.addGenerated(StringLTrimDescriptor.FACTORY);
+        fc.addGenerated(StringRTrimDescriptor.FACTORY);
+        fc.addGenerated(StringTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringLTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringRTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringPositionDescriptor.FACTORY);
+        fc.addGenerated(StringRepeatDescriptor.FACTORY);
+        fc.addGenerated(StringSplitDescriptor.FACTORY);
 
         // Constructors
-        functionsToInjectUnkownHandling.add(ABooleanConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ABinaryHexStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ABinaryBase64StringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt8ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt16ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt32ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt64ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AFloatConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADoubleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APointConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APoint3DConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ALineConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APolygonConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ACircleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ARectangleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ATimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADateConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADateTimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AYearMonthDurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADayTimeDurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AUUIDFromStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromDateConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABooleanConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABinaryHexStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABinaryBase64StringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt8ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt16ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt32ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt64ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AFloatConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADoubleConstructorDescriptor.FACTORY);
+        fc.addGenerated(APointConstructorDescriptor.FACTORY);
+        fc.addGenerated(APoint3DConstructorDescriptor.FACTORY);
+        fc.addGenerated(ALineConstructorDescriptor.FACTORY);
+        fc.addGenerated(APolygonConstructorDescriptor.FACTORY);
+        fc.addGenerated(ACircleConstructorDescriptor.FACTORY);
+        fc.addGenerated(ARectangleConstructorDescriptor.FACTORY);
+        fc.addGenerated(ATimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADateConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADateTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(AYearMonthDurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADayTimeDurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(AUUIDFromStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromDateConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
 
         // Spatial
-        functionsToInjectUnkownHandling.add(CreatePointDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateLineDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreatePolygonDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateCircleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateRectangleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialAreaDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialDistanceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialIntersectDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateMBRDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialCellDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PointXCoordinateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(PointYCoordinateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(CircleRadiusAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(CircleCenterAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(LineRectanglePolygonAccessor.FACTORY);
+        fc.addGenerated(CreatePointDescriptor.FACTORY);
+        fc.addGenerated(CreateLineDescriptor.FACTORY);
+        fc.addGenerated(CreatePolygonDescriptor.FACTORY);
+        fc.addGenerated(CreateCircleDescriptor.FACTORY);
+        fc.addGenerated(CreateRectangleDescriptor.FACTORY);
+        fc.addGenerated(SpatialAreaDescriptor.FACTORY);
+        fc.addGenerated(SpatialDistanceDescriptor.FACTORY);
+        fc.addGenerated(SpatialIntersectDescriptor.FACTORY);
+        fc.addGenerated(CreateMBRDescriptor.FACTORY);
+        fc.addGenerated(SpatialCellDescriptor.FACTORY);
+        fc.addGenerated(PointXCoordinateAccessor.FACTORY);
+        fc.addGenerated(PointYCoordinateAccessor.FACTORY);
+        fc.addGenerated(CircleRadiusAccessor.FACTORY);
+        fc.addGenerated(CircleCenterAccessor.FACTORY);
+        fc.addGenerated(LineRectanglePolygonAccessor.FACTORY);
 
         // fuzzyjoin function
-        functionsToInjectUnkownHandling.add(PrefixLenJaccardDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(WordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(HashedWordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CountHashedWordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(HashedGramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CountHashedGramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceCheckDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceStringIsFilterableDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceListIsFilterableDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardCheckDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardSortedDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardSortedCheckDescriptor.FACTORY);
+        fc.addGenerated(PrefixLenJaccardDescriptor.FACTORY);
+        fc.addGenerated(WordTokensDescriptor.FACTORY);
+        fc.addGenerated(HashedWordTokensDescriptor.FACTORY);
+        fc.addGenerated(CountHashedWordTokensDescriptor.FACTORY);
+        fc.addGenerated(GramTokensDescriptor.FACTORY);
+        fc.addGenerated(HashedGramTokensDescriptor.FACTORY);
+        fc.addGenerated(CountHashedGramTokensDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceCheckDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceStringIsFilterableDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceListIsFilterableDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceContainsDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardCheckDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardSortedDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardSortedCheckDescriptor.FACTORY);
 
         // full-text function
-        functionsToInjectUnkownHandling.add(FullTextContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FullTextContainsWithoutOptionDescriptor.FACTORY);
+        fc.addGenerated(FullTextContainsDescriptor.FACTORY);
+        fc.addGenerated(FullTextContainsWithoutOptionDescriptor.FACTORY);
 
         // Record functions.
-        functionsToInjectUnkownHandling.add(GetRecordFieldsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetRecordFieldValueDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DeepEqualityDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordMergeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordAddFieldsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordRemoveFieldsDescriptor.FACTORY);
+        fc.addGenerated(GetRecordFieldsDescriptor.FACTORY);
+        fc.addGenerated(GetRecordFieldValueDescriptor.FACTORY);
+        fc.addGenerated(DeepEqualityDescriptor.FACTORY);
+        fc.addGenerated(RecordMergeDescriptor.FACTORY);
+        fc.addGenerated(RecordAddFieldsDescriptor.FACTORY);
+        fc.addGenerated(RecordRemoveFieldsDescriptor.FACTORY);
 
         // Spatial and temporal type accessors
-        functionsToInjectUnkownHandling.add(TemporalYearAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMonthAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalDayAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalHourAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMinuteAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalSecondAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMillisecondAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartDateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndDateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartTimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndTimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartDatetimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndDatetimeAccessor.FACTORY);
+        fc.addGenerated(TemporalYearAccessor.FACTORY);
+        fc.addGenerated(TemporalMonthAccessor.FACTORY);
+        fc.addGenerated(TemporalDayAccessor.FACTORY);
+        fc.addGenerated(TemporalHourAccessor.FACTORY);
+        fc.addGenerated(TemporalMinuteAccessor.FACTORY);
+        fc.addGenerated(TemporalSecondAccessor.FACTORY);
+        fc.addGenerated(TemporalMillisecondAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartDateAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndDateAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartTimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndTimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartDatetimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndDatetimeAccessor.FACTORY);
 
         // Temporal functions
-        functionsToInjectUnkownHandling.add(UnixTimeFromDateInDaysDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromDatetimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromDatetimeInSecsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DateFromDatetimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(TimeFromDatetimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CalendarDuartionFromDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AdjustTimeForTimeZoneDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalBeforeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalAfterDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalMeetsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalMetByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalOverlapsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalOverlappedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(OverlapDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalStartsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalStartedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalCoversDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalCoveredByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalEndsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalEndedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromMillisecondsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromMonthsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(YearMonthDurationGreaterThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(YearMonthDurationLessThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayTimeDurationGreaterThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayTimeDurationLessThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(MonthsFromYearMonthDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationEqualDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetYearMonthDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetDayTimeDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalBinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(OverlapBinsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayOfWeekDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetOverlappingIntervalDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromIntervalDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDateInDaysDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDatetimeInMsDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDatetimeInSecsDescriptor.FACTORY);
+        fc.addGenerated(DateFromUnixTimeInDaysDescriptor.FACTORY);
+        fc.addGenerated(DateFromDatetimeDescriptor.FACTORY);
+        fc.addGenerated(TimeFromUnixTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(TimeFromDatetimeDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromDateAndTimeDescriptor.FACTORY);
+        fc.addGenerated(CalendarDurationFromDateTimeDescriptor.FACTORY);
+        fc.addGenerated(CalendarDuartionFromDateDescriptor.FACTORY);
+        fc.addGenerated(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
+        fc.addGenerated(AdjustTimeForTimeZoneDescriptor.FACTORY);
+        fc.addGenerated(IntervalBeforeDescriptor.FACTORY);
+        fc.addGenerated(IntervalAfterDescriptor.FACTORY);
+        fc.addGenerated(IntervalMeetsDescriptor.FACTORY);
+        fc.addGenerated(IntervalMetByDescriptor.FACTORY);
+        fc.addGenerated(IntervalOverlapsDescriptor.FACTORY);
+        fc.addGenerated(IntervalOverlappedByDescriptor.FACTORY);
+        fc.addGenerated(OverlapDescriptor.FACTORY);
+        fc.addGenerated(IntervalStartsDescriptor.FACTORY);
+        fc.addGenerated(IntervalStartedByDescriptor.FACTORY);
+        fc.addGenerated(IntervalCoversDescriptor.FACTORY);
+        fc.addGenerated(IntervalCoveredByDescriptor.FACTORY);
+        fc.addGenerated(IntervalEndsDescriptor.FACTORY);
+        fc.addGenerated(IntervalEndedByDescriptor.FACTORY);
+        fc.addGenerated(DurationFromMillisecondsDescriptor.FACTORY);
+        fc.addGenerated(DurationFromMonthsDescriptor.FACTORY);
+        fc.addGenerated(YearMonthDurationGreaterThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(YearMonthDurationLessThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(DayTimeDurationGreaterThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(DayTimeDurationLessThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(MonthsFromYearMonthDurationDescriptor.FACTORY);
+        fc.addGenerated(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
+        fc.addGenerated(DurationEqualDescriptor.FACTORY);
+        fc.addGenerated(GetYearMonthDurationDescriptor.FACTORY);
+        fc.addGenerated(GetDayTimeDurationDescriptor.FACTORY);
+        fc.addGenerated(IntervalBinDescriptor.FACTORY);
+        fc.addGenerated(OverlapBinsDescriptor.FACTORY);
+        fc.addGenerated(DayOfWeekDescriptor.FACTORY);
+        fc.addGenerated(ParseDateDescriptor.FACTORY);
+        fc.addGenerated(ParseTimeDescriptor.FACTORY);
+        fc.addGenerated(ParseDateTimeDescriptor.FACTORY);
+        fc.addGenerated(PrintDateDescriptor.FACTORY);
+        fc.addGenerated(PrintTimeDescriptor.FACTORY);
+        fc.addGenerated(PrintDateTimeDescriptor.FACTORY);
+        fc.addGenerated(GetOverlappingIntervalDescriptor.FACTORY);
+        fc.addGenerated(DurationFromIntervalDescriptor.FACTORY);
 
         // Type functions.
-        functionsToInjectUnkownHandling.add(IsBooleanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsNumberDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsArrayDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsObjectDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToBooleanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToDoubleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToBigIntDescriptor.FACTORY);
+        fc.addGenerated(IsBooleanDescriptor.FACTORY);
+        fc.addGenerated(IsNumberDescriptor.FACTORY);
+        fc.addGenerated(IsStringDescriptor.FACTORY);
+        fc.addGenerated(IsArrayDescriptor.FACTORY);
+        fc.addGenerated(IsObjectDescriptor.FACTORY);
+        fc.addGenerated(ToBooleanDescriptor.FACTORY);
+        fc.addGenerated(ToStringDescriptor.FACTORY);
+        fc.addGenerated(ToDoubleDescriptor.FACTORY);
+        fc.addGenerated(ToBigIntDescriptor.FACTORY);
 
         // Cast function
-        functionsToInjectUnkownHandling.add(CastTypeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CastTypeLaxDescriptor.FACTORY);
+        fc.addGenerated(CastTypeDescriptor.FACTORY);
+        fc.addGenerated(CastTypeLaxDescriptor.FACTORY);
 
         // Record function
-        functionsToInjectUnkownHandling.add(RecordPairsDescriptor.FACTORY);
+        fc.addGenerated(RecordPairsDescriptor.FACTORY);
 
-        List<IFunctionDescriptorFactory> generatedFactories = new ArrayList<>();
-        for (IFunctionDescriptorFactory factory : functionsToInjectUnkownHandling) {
-            generatedFactories
-                    .add(getGeneratedFunctionDescriptorFactory(factory.createFunctionDescriptor().getClass()));
-        }
-        temp.addAll(generatedFactories);
+        return fc;
     }
 
-    public static List<IFunctionDescriptorFactory> getFunctionDescriptorFactories() {
-        return temp;
+    public List<IFunctionDescriptorFactory> getFunctionDescriptorFactories() {
+        return descriptorFactories;
     }
 
     /**
@@ -739,8 +745,5 @@
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
-    }
-
-    private FunctionCollection() {
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
new file mode 100644
index 0000000..117ca31
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
@@ -0,0 +1,70 @@
+/*
+ * 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.functions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public final class FunctionManager implements IFunctionManager {
+
+    private final Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functions;
+
+    private final Map<FunctionIdentifier, IFunctionTypeInferer> typeInferers;
+
+    public FunctionManager(FunctionCollection functionCollection) {
+        Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functionsMap = new HashMap<>();
+        Map<FunctionIdentifier, IFunctionTypeInferer> typeInferersMap = new HashMap<>();
+
+        for (IFunctionDescriptorFactory descriptorFactory : functionCollection.getFunctionDescriptorFactories()) {
+            FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
+            functionsMap.put(new Pair<>(fid, fid.getArity()), descriptorFactory);
+            IFunctionTypeInferer typeInferer = descriptorFactory.createFunctionTypeInferer();
+            if (typeInferer != null) {
+                typeInferersMap.put(fid, typeInferer);
+            }
+        }
+
+        this.functions = functionsMap;
+        this.typeInferers = typeInferersMap;
+    }
+
+    @Override
+    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
+        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
+        IFunctionDescriptorFactory factory = functions.get(key);
+        if (factory == null) {
+            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
+        }
+        return factory.createFunctionDescriptor();
+    }
+
+    @Override
+    public IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid) {
+        return typeInferers.get(fid);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
deleted file mode 100644
index 52b61bb..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.functions;
-
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.functions.IFunctionManager;
-
-public final class FunctionManagerHolder {
-    private static final IFunctionManager functionManager = createFunctionManager();
-
-    public static IFunctionManager getFunctionManager() {
-        return functionManager;
-    }
-
-    private static IFunctionManager createFunctionManager() {
-        FunctionManagerImpl mgr = new FunctionManagerImpl();
-        for (IFunctionDescriptorFactory fdFactory : FunctionCollection.getFunctionDescriptorFactories()) {
-            mgr.registerFunction(fdFactory);
-        }
-        return mgr;
-    }
-
-    private FunctionManagerHolder() {
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
deleted file mode 100644
index 4056955..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.runtime.functions;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.functions.IFunctionManager;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-
-final class FunctionManagerImpl implements IFunctionManager {
-    private final Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functions;
-
-    FunctionManagerImpl() {
-        functions = new HashMap<>();
-    }
-
-    @Override
-    public synchronized IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
-        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
-        IFunctionDescriptorFactory factory = functions.get(key);
-        if (factory == null) {
-            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
-        }
-        return factory.createFunctionDescriptor();
-    }
-
-    @Override
-    public synchronized void registerFunction(IFunctionDescriptorFactory descriptorFactory) {
-        FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        functions.put(new Pair<>(fid, fid.getArity()), descriptorFactory);
-    }
-
-    @Override
-    public synchronized void unregisterFunction(IFunctionDescriptorFactory descriptorFactory) {
-        FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
-        functions.remove(key);
-    }
-
-    @Override
-    public synchronized Iterator<IFunctionDescriptorFactory> iterator() {
-        return new ArrayList<>(functions.values()).iterator();
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
new file mode 100644
index 0000000..3bfd522
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -0,0 +1,268 @@
+/*
+ * 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.functions;
+
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public final class FunctionTypeInferers {
+
+    private FunctionTypeInferers() {
+    }
+
+    public static final IFunctionTypeInferer SET_EXPRESSION_TYPE = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            fd.setImmutableStates(context.getType(expr));
+        }
+    };
+
+    public static final class CastTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            IAType rt = TypeCastUtils.getRequiredType(funcExpr);
+            IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+            fd.setImmutableStates(rt, it);
+        }
+    }
+
+    public static final class DeepEqualityTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
+            fd.setImmutableStates(type0, type1);
+        }
+    }
+
+    public static final class FieldAccessByIndexTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            switch (t.getTypeTag()) {
+                case OBJECT: {
+                    fd.setImmutableStates(t);
+                    break;
+                }
+                case UNION: {
+                    AUnionType unionT = (AUnionType) t;
+                    if (unionT.isUnknownableType()) {
+                        IAType t2 = unionT.getActualType();
+                        if (t2.getTypeTag() == ATypeTag.OBJECT) {
+                            fd.setImmutableStates(t2);
+                            break;
+                        }
+                    }
+                    throw new NotImplementedException("field-access-by-index for data of type " + t);
+                }
+                default: {
+                    throw new NotImplementedException("field-access-by-index for data of type " + t);
+                }
+            }
+        }
+    }
+
+    public static final class FieldAccessNestedTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            AOrderedList fieldPath =
+                    (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce.getArguments().get(1).getValue())
+                            .getValue()).getObject());
+            List<String> listFieldPath = new ArrayList<>();
+            for (int i = 0; i < fieldPath.size(); i++) {
+                listFieldPath.add(((AString) fieldPath.getItem(i)).getStringValue());
+            }
+
+            switch (t.getTypeTag()) {
+                case OBJECT: {
+                    fd.setImmutableStates(t, listFieldPath);
+                    break;
+                }
+                case ANY:
+                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
+                    break;
+                default: {
+                    throw new NotImplementedException("field-access-nested for data of type " + t);
+                }
+            }
+        }
+    }
+
+    public static final class GetRecordFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("get-record-fields for data of type " + t);
+            }
+        }
+    }
+
+    public static final class GetRecordFieldValueTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("get-record-field-value for data of type " + t);
+            }
+        }
+    }
+
+    public static final class OpenRecordConstructorTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            ARecordType rt = (ARecordType) context.getType(expr);
+            fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
+        }
+
+        private boolean[] computeOpenFields(AbstractFunctionCallExpression expr, ARecordType recType) {
+            int n = expr.getArguments().size() / 2;
+            boolean[] open = new boolean[n];
+            for (int i = 0; i < n; i++) {
+                Mutable<ILogicalExpression> argRef = expr.getArguments().get(2 * i);
+                ILogicalExpression arg = argRef.getValue();
+                open[i] = true;
+                final String fn = ConstantExpressionUtil.getStringConstant(arg);
+                if (fn != null) {
+                    for (String s : recType.getFieldNames()) {
+                        if (s.equals(fn)) {
+                            open[i] = false;
+                            break;
+                        }
+                    }
+                }
+            }
+            return open;
+        }
+    }
+
+    public static final class RecordAddFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            ILogicalExpression listExpr = f.getArguments().get(1).getValue();
+            IAType type1 = (IAType) context.getType(listExpr);
+            if (type0.getTypeTag().equals(ATypeTag.ANY)) {
+                type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            if (type1.getTypeTag().equals(ATypeTag.ANY)) {
+                type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+
+    public static final class RecordMergeTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+
+    public static final class RecordPairsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("record-fields with data of type " + t);
+            }
+        }
+    }
+
+    public static final class RecordRemoveFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            ILogicalExpression le = f.getArguments().get(1).getValue();
+            IAType type1 = (IAType) context.getType(le);
+            if (type0.getTypeTag().equals(ATypeTag.ANY)) {
+                type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            if (type1.getTypeTag().equals(ATypeTag.ANY)) {
+                type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+}

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1381/ (3/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/1410/ (2/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/2300/ (9/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/2109/ (10/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins, Ian Maxon,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2118

to look at the new patch set (#5).

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................

[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
60 files changed, 1,189 insertions(+), 944 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/18/2118/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1418/ (4/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2049/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/1382/ (1/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1742/ (6/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1744/ (1/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1697/ (5/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/2085/ (8/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/1362/ (9/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4424/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/2053/ (7/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1370/ (4/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/1987/ (4/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5: Contrib-1

Analytics Compatibility Tests Failed Build 2884
: UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins, Ian Maxon,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2118

to look at the new patch set (#4).

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................

[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
60 files changed, 1,189 insertions(+), 944 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/18/2118/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/2259/ (7/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5465/ (6/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6959/ (3/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/1976/ (8/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/2080/ (10/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6924/ (8/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/2024/ (5/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1416/ (7/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/1417/ (7/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/2021/ (1/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Xikui Wang (Code Review)" <do...@asterixdb.incubator.apache.org>.
Xikui Wang has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

(1 comment)

one minor comment. other than that, i think it's a +2 from me.

https://asterix-gerrit.ics.uci.edu/#/c/2118/2/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
File asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java:

Line 885:         ISerializerDeserializer<?> payloadSerde = getFormat().getSerdeProvider().getSerializerDeserializer(itemType);
although it's not part of this change, maybe you want to change the method name to getDataFormat() which is more informative...


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5417/ (6/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1741/ (5/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/1408/ (2/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/2022/ (5/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/1407/ (2/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4436/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/1371/ (1/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6958/ (6/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1415/ (3/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4467/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

LGTM. One minor comment: for the future reference, can you put a Javadoc comment before new classes or interfaces? (e.g., FunctionTypeInferers)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5428/ (5/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5463/ (4/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2063/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/1373/ (9/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6913/ (2/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4436/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/2042/ (3/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1708/ (2/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/2069/ (10/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4424/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

(2 comments)

@Taewoo, I added javadocs for new classes as you requested.

https://asterix-gerrit.ics.uci.edu/#/c/2118/2/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
File asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java:

PS2, Line 76: Set<ExtensionId> extensionIds = new HashSet<>()
> So this is just to check that we're not getting duplicate extensions?
Yes. There used to be a field called 'extensions' which was a Map<>, but the values were not used. So I refactored it into a local Set<>


https://asterix-gerrit.ics.uci.edu/#/c/2118/2/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
File asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java:

Line 885:         ISerializerDeserializer<?> payloadSerde = getDataFormat().getSerdeProvider().getSerializerDeserializer(itemType);
> although it's not part of this change, maybe you want to change the method 
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins, Ian Maxon,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2118

to look at the new patch set (#3).

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................

[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
60 files changed, 1,184 insertions(+), 941 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/18/2118/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5462/ (4/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4467/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Ian Maxon (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ian Maxon has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2: Code-Review+1

(1 comment)

Seems good to me.

https://asterix-gerrit.ics.uci.edu/#/c/2118/2/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
File asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java:

PS2, Line 76: Set<ExtensionId> extensionIds = new HashSet<>()
So this is just to check that we're not getting duplicate extensions?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2049/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/1418/ (1/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2063/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has submitted this change and it was merged.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2118
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <im...@apache.org>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
60 files changed, 1,189 insertions(+), 944 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Ian Maxon: Looks good to me, approved

Objections:
  Anon. E. Moose #1000171: 
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
index 4803691..fb74475 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.common.api.IExtension;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.om.functions.IFunctionManager;
 
 /**
  * An interface for language extensions
@@ -38,4 +39,8 @@
     }
 
     ILangCompilationProvider getLangCompilationProvider(Language lang);
+
+    default IFunctionManager getFunctionManager() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 9f88bb6..812f3c9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -23,11 +23,12 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
-import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.runtime.formats.FormatUtils;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -40,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -51,9 +53,10 @@
 
 public class QueryLogicalExpressionJobGen implements ILogicalExpressionJobGen {
 
-    public static final QueryLogicalExpressionJobGen INSTANCE = new QueryLogicalExpressionJobGen();
+    private final IFunctionManager functionManager;
 
-    private QueryLogicalExpressionJobGen() {
+    public QueryLogicalExpressionJobGen(IFunctionManager functionManager) {
+        this.functionManager = functionManager;
     }
 
     @Override
@@ -61,7 +64,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
+        IFunctionDescriptor fd = resolveFunction(expr, env);
         switch (fd.getFunctionDescriptorTag()) {
             case SERIALAGGREGATE:
                 return null;
@@ -79,7 +82,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return getFunctionDescriptor(expr, env, context).createRunningAggregateEvaluatorFactory(args);
+        return resolveFunction(expr, env).createRunningAggregateEvaluatorFactory(args);
     }
 
     @Override
@@ -87,22 +90,22 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return getFunctionDescriptor(expr, env, context).createUnnestingEvaluatorFactory(args);
+        return resolveFunction(expr, env).createUnnestingEvaluatorFactory(args);
     }
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
             IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        IScalarEvaluatorFactory copyEvaluatorFactory = null;
+        IScalarEvaluatorFactory copyEvaluatorFactory;
         switch (expr.getExpressionTag()) {
             case VARIABLE: {
                 VariableReferenceExpression v = (VariableReferenceExpression) expr;
-                copyEvaluatorFactory = createVariableEvaluatorFactory(v, inputSchemas, context);
+                copyEvaluatorFactory = createVariableEvaluatorFactory(v, inputSchemas);
                 return copyEvaluatorFactory;
             }
             case CONSTANT: {
                 ConstantExpression c = (ConstantExpression) expr;
-                copyEvaluatorFactory = createConstantEvaluatorFactory(c, inputSchemas, context);
+                copyEvaluatorFactory = createConstantEvaluatorFactory(c, context);
                 return copyEvaluatorFactory;
             }
             case FUNCTION_CALL: {
@@ -117,7 +120,7 @@
     }
 
     private IScalarEvaluatorFactory createVariableEvaluatorFactory(VariableReferenceExpression expr,
-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+            IOperatorSchema[] inputSchemas) throws AlgebricksException {
         LogicalVariable variable = expr.getVariableReference();
         for (IOperatorSchema scm : inputSchemas) {
             int pos = scm.findVariable(variable);
@@ -132,22 +135,17 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = null;
-        if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
-            IDataFormat format = FormatUtils.getDefaultFormat();
-            fd = format.resolveFunction(expr, env);
-        } else {
-            ICcApplicationContext appCtx = (ICcApplicationContext) context.getAppContext();
-            fd = ExternalFunctionDescriptorProvider
-                    .getExternalFunctionDescriptor((IExternalFunctionInfo) expr.getFunctionInfo(), appCtx);
-        }
+        IFunctionDescriptor fd = expr.getFunctionInfo() instanceof IExternalFunctionInfo
+                ? ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor(
+                        (IExternalFunctionInfo) expr.getFunctionInfo(), (ICcApplicationContext) context.getAppContext())
+                : resolveFunction(expr, env);
         return fd.createEvaluatorFactory(args);
     }
 
-    private IScalarEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr,
-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        IDataFormat format = FormatUtils.getDefaultFormat();
-        return format.getConstantEvalFactory(expr.getValue());
+    private IScalarEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr, JobGenContext context)
+            throws AlgebricksException {
+        MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+        return metadataProvider.getDataFormat().getConstantEvalFactory(expr.getValue());
     }
 
     private IScalarEvaluatorFactory[] codegenArguments(AbstractFunctionCallExpression expr,
@@ -168,14 +166,14 @@
             AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
             JobGenContext context) throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
+        IFunctionDescriptor fd = resolveFunction(expr, env);
 
         switch (fd.getFunctionDescriptorTag()) {
             case AGGREGATE: {
                 if (BuiltinFunctions.isAggregateFunctionSerializable(fd.getIdentifier())) {
                     AggregateFunctionCallExpression serialAggExpr = BuiltinFunctions
                             .makeSerializableAggregateFunctionExpression(fd.getIdentifier(), expr.getArguments());
-                    IFunctionDescriptor afdd = getFunctionDescriptor(serialAggExpr, env, context);
+                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env);
                     return afdd.createSerializableAggregateEvaluatorFactory(args);
                 } else {
                     throw new AlgebricksException(
@@ -194,10 +192,14 @@
         }
     }
 
-    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
-            JobGenContext context) throws AlgebricksException {
-        IFunctionDescriptor fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
+    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env)
+            throws AlgebricksException {
+        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
+        IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
+        IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
+        if (fnTypeInfer != null) {
+            fnTypeInfer.infer(expr, fd, env);
+        }
         return fd;
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 5ef41c4..f7695ea 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -36,9 +36,11 @@
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -125,12 +127,13 @@
     private static final IOperatorSchema[] _emptySchemas = new IOperatorSchema[] {};
 
     public ConstantFoldingRule(ICcApplicationContext appCtx) {
-        jobGenCtx = new JobGenContext(null, null, appCtx, SerializerDeserializerProvider.INSTANCE,
+        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
                 BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
                 BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
                 BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, null,
-                new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE), ExpressionTypeComputer.INSTANCE,
-                null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
+                ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 8290446..2078288 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -263,7 +263,7 @@
         builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
         builder.setLogicalRewrites(ruleSetFactory.getLogicalRewrites(metadataProvider.getApplicationContext()));
         builder.setPhysicalRewrites(ruleSetFactory.getPhysicalRewrites(metadataProvider.getApplicationContext()));
-        IDataFormat format = metadataProvider.getFormat();
+        IDataFormat format = metadataProvider.getDataFormat();
         ICompilerFactory compilerFactory = builder.create();
         builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
         builder.setIMergeAggregationExpressionFactory(new MergeAggregationExpressionFactory());
@@ -321,7 +321,8 @@
         builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
         builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
         builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE));
+        builder.setExpressionRuntimeProvider(
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
         builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
         builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
         builder.setMissingWriterFactory(format.getMissingWriterFactory());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 0c6b2cc..768416d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -18,9 +18,9 @@
  */
 package org.apache.asterix.app.cc;
 
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.asterix.algebra.base.ILangExtension;
@@ -35,22 +35,25 @@
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.runtime.functions.FunctionCollection;
+import org.apache.asterix.runtime.functions.FunctionManager;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.utils.ExtensionUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * AsterixDB's implementation of {@code IAlgebraExtensionManager} which takes care of
- * initializing extensions for App and Compilation purposes
+ * AsterixDB's implementation of {@code IAlgebraExtensionManager} and {@code IFunctionExtensionManager}
+ * which takes care of initializing extensions for App and Compilation purposes
  */
-public class CCExtensionManager implements IAlgebraExtensionManager {
-
-    private final Map<ExtensionId, IExtension> extensions = new HashMap<>();
+public class CCExtensionManager implements IAlgebraExtensionManager, IFunctionExtensionManager {
 
     private final IStatementExecutorExtension statementExecutorExtension;
     private final ILangCompilationProvider aqlCompilationProvider;
     private final ILangCompilationProvider sqlppCompilationProvider;
+    private final IFunctionManager functionManager;
     private transient IStatementExecutorFactory statementExecutorFactory;
 
     /**
@@ -67,15 +70,16 @@
             throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
         Pair<ExtensionId, ILangCompilationProvider> aqlcp = null;
         Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
+        Pair<ExtensionId, IFunctionManager> fm = null;
         IStatementExecutorExtension see = null;
         if (list != null) {
+            Set<ExtensionId> extensionIds = new HashSet<>();
             for (AsterixExtension extensionConf : list) {
                 IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
                 extension.configure(extensionConf.getArgs());
-                if (extensions.containsKey(extension.getId())) {
+                if (!extensionIds.add(extension.getId())) {
                     throw new RuntimeDataException(ErrorCode.EXTENSION_ID_CONFLICT, extension.getId());
                 }
-                extensions.put(extension.getId(), extension);
                 switch (extension.getExtensionKind()) {
                     case STATEMENT_EXECUTOR:
                         see = ExtensionUtil.extendStatementExecutor(see, (IStatementExecutorExtension) extension);
@@ -84,6 +88,7 @@
                         ILangExtension le = (ILangExtension) extension;
                         aqlcp = ExtensionUtil.extendLangCompilationProvider(Language.AQL, aqlcp, le);
                         sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
+                        fm = ExtensionUtil.extendFunctionManager(fm, le);
                         break;
                     default:
                         break;
@@ -93,6 +98,8 @@
         this.statementExecutorExtension = see;
         this.aqlCompilationProvider = aqlcp == null ? new AqlCompilationProvider() : aqlcp.second;
         this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
+        this.functionManager =
+                fm == null ? new FunctionManager(FunctionCollection.createDefaultFunctionCollection()) : fm.second;
     }
 
     /** @deprecated use getStatementExecutorFactory instead */
@@ -117,4 +124,9 @@
             default: throw new IllegalArgumentException(String.valueOf(lang));
         }
     }
+
+    @Override
+    public IFunctionManager getFunctionManager() {
+        return functionManager;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
index 07eed0d..380c5a9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.metadata.api.IMetadataExtension;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
@@ -52,18 +53,39 @@
      */
     public static Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
             Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws RuntimeDataException {
-        if (cp != null && le.getLangCompilationProvider(lang) != null) {
+        ILangCompilationProvider lecp = le.getLangCompilationProvider(lang);
+        if (cp != null && lecp != null) {
             throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), cp.first,
                     lang.toString());
         }
-        return (le.getLangCompilationProvider(lang) != null)
-                ? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
+        return lecp != null ? new Pair<>(le.getId(), lecp) : cp;
+    }
+
+    /**
+     * Validate no extension conflict and return function manager extension
+     *
+     * @param fm
+     *            place holder for function manager extension
+     * @param le
+     *            user defined extension
+     * @return the user defined extension
+     * @throws RuntimeDataException
+     *             if extension conflict was detected
+     */
+    public static Pair<ExtensionId, IFunctionManager> extendFunctionManager(Pair<ExtensionId, IFunctionManager> fm,
+            ILangExtension le) throws RuntimeDataException {
+        IFunctionManager lefm = le.getFunctionManager();
+        if (fm != null && lefm != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), fm.first,
+                    IFunctionManager.class.getSimpleName());
+        }
+        return lefm != null ? new Pair<>(le.getId(), lefm) : fm;
     }
 
     /**
      * Validate no extension conflict and return statement executor extension
      *
-     * @param qte
+     * @param see
      *            place holder for statement executor extension
      * @param extension
      *            user defined extension
@@ -71,10 +93,10 @@
      * @throws RuntimeDataException
      *             if extension conflict was detected
      */
-    public static IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
+    public static IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension see,
             IStatementExecutorExtension extension) throws RuntimeDataException {
-        if (qte != null) {
-            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
+        if (see != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, see.getId(), extension.getId(),
                     IStatementExecutorFactory.class.getSimpleName());
         }
         return extension;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
index 3c54c7c..4bcee04 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
@@ -43,7 +43,8 @@
 
     @Test
     public void test() throws Exception {
-        List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+        List<IFunctionDescriptorFactory> functions =
+                FunctionCollection.createDefaultFunctionCollection().getFunctionDescriptorFactories();
         int testedFunctions = 0;
         for (IFunctionDescriptorFactory func : functions) {
             String className = func.getClass().getName();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
index f5e7402..3b33868 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
@@ -43,7 +43,8 @@
 
     @Test
     public void test() throws Exception {
-        List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+        List<IFunctionDescriptorFactory> functions =
+                FunctionCollection.createDefaultFunctionCollection().getFunctionDescriptorFactories();
         int testedFunctions = 0;
         for (IFunctionDescriptorFactory func : functions) {
             String className = func.getClass().getName();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index b1191ec..40d4f6a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -45,6 +45,9 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.lock.MetadataLockManager;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
+import org.apache.asterix.runtime.functions.FunctionCollection;
+import org.apache.asterix.runtime.functions.FunctionManager;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.active.TestEventsListener.Behavior;
 import org.apache.asterix.test.base.TestMethodTracer;
@@ -82,6 +85,7 @@
     static CcApplicationContext appCtx;
     static IStatementExecutor statementExecutor;
     static IHyracksClientConnection hcc;
+    static IFunctionExtensionManager functionExtensionManager;
     static MetadataProvider metadataProvider;
     static IStorageComponentProvider componentProvider;
     static JobIdFactory jobIdFactory;
@@ -121,6 +125,10 @@
         Mockito.when(ccServiceCtx.getControllerService()).thenReturn(ccService);
         Mockito.when(ccService.getExecutor()).thenReturn(executor);
         locations = new AlgebricksAbsolutePartitionConstraint(nodes);
+        functionExtensionManager = Mockito.mock(IFunctionExtensionManager.class);
+        Mockito.when(functionExtensionManager.getFunctionManager())
+                .thenReturn(new FunctionManager(FunctionCollection.createDefaultFunctionCollection()));
+        Mockito.when(appCtx.getExtensionManager()).thenReturn(functionExtensionManager);
         metadataProvider = new MetadataProvider(appCtx, null);
         clusterController = new TestClusterControllerActor("CC", handler, allDatasets);
         nodeControllers = new TestNodeControllerActor[2];
@@ -1449,6 +1457,10 @@
             CcApplicationContext ccAppCtx = Mockito.mock(CcApplicationContext.class);
             IStatementExecutor statementExecutor = Mockito.mock(IStatementExecutor.class);
             IHyracksClientConnection hcc = Mockito.mock(IHyracksClientConnection.class);
+            IFunctionExtensionManager functionExtensionManager = Mockito.mock(IFunctionExtensionManager.class);
+            Mockito.when(functionExtensionManager.getFunctionManager())
+                    .thenReturn(new FunctionManager(FunctionCollection.createDefaultFunctionCollection()));
+            Mockito.when(ccAppCtx.getExtensionManager()).thenReturn(functionExtensionManager);
             Mockito.when(ccAppCtx.getActiveNotificationHandler()).thenReturn(handler);
             Mockito.when(ccAppCtx.getMetadataLockManager()).thenReturn(lockManager);
             Mockito.when(ccAppCtx.getServiceContext()).thenReturn(ccServiceCtx);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index e2b1761..3f3a27f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -31,7 +31,6 @@
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -107,8 +106,7 @@
                 ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
                 IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
                         edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, null);
-                return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
-                        NonTaggedDataFormat.INSTANCE);
+                return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory);
             case INTERNAL:
                 DataSourceId id = getId();
                 String dataverseName = id.getDataverseName();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 0b6608c..325d23b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -33,7 +33,6 @@
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -170,7 +169,7 @@
         try {
             ARecordType feedOutputType = (ARecordType) itemType;
             ISerializerDeserializer payloadSerde =
-                    NonTaggedDataFormat.INSTANCE.getSerdeProvider().getSerializerDeserializer(feedOutputType);
+                    metadataProvider.getDataFormat().getSerdeProvider().getSerializerDeserializer(feedOutputType);
             ArrayList<ISerializerDeserializer> serdes = new ArrayList<>();
             serdes.add(payloadSerde);
             if (metaItemType != null) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 9271f332..2f94ad7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -57,6 +57,7 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -78,6 +79,7 @@
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
@@ -143,6 +145,7 @@
     private final ICcApplicationContext appCtx;
     private final IStorageComponentProvider storageComponentProvider;
     private final StorageProperties storageProperties;
+    private final IFunctionManager functionManager;
     private final Dataverse defaultDataverse;
     private final LockList locks;
     private final Map<String, String> config;
@@ -164,6 +167,7 @@
         this.defaultDataverse = defaultDataverse == null ? MetadataBuiltinEntities.DEFAULT_DATAVERSE : defaultDataverse;
         this.storageComponentProvider = appCtx.getStorageComponentProvider();
         storageProperties = appCtx.getStorageProperties();
+        functionManager = ((IFunctionExtensionManager) appCtx.getExtensionManager()).getFunctionManager();
         locks = new LockList();
         config = new HashMap<>();
     }
@@ -259,7 +263,11 @@
         return isTemporaryDatasetWriteJob;
     }
 
-    public IDataFormat getFormat() {
+    public IFunctionManager getFunctionManager() {
+        return functionManager;
+    }
+
+    public IDataFormat getDataFormat() {
         return FormatUtils.getDefaultFormat();
     }
 
@@ -868,13 +876,14 @@
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
-            JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory, IDataFormat format)
+            JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory)
             throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.OBJECT) {
             throw new AlgebricksException("Can only scan datasets of records.");
         }
 
-        ISerializerDeserializer<?> payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
+        ISerializerDeserializer<?> payloadSerde =
+                getDataFormat().getSerdeProvider().getSerializerDeserializer(itemType);
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         ExternalScanOperatorDescriptor dataScanner =
@@ -1501,7 +1510,7 @@
             // Generate Output Record format
             ISerializerDeserializer<?>[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
             ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
-            ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
+            ISerializerDeserializerProvider serdeProvider = getDataFormat().getSerdeProvider();
 
             // The order of the output record: propagated variables (including
             // PK and SK), token, and number of token.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 9156b0f..8e1c34d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -730,7 +730,7 @@
         ISerializerDeserializer[] primaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + 1 + (hasMetaPart() ? 1 : 0)];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1 + (hasMetaPart() ? 1 : 0)];
-        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
         List<Integer> indicators = null;
         if (hasMetaPart()) {
             indicators = ((InternalDatasetDetails) getDatasetDetails()).getKeySourceIndicator();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index e4a6ca8..53c3dc0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -41,6 +41,7 @@
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.metadata.MetadataManager;
@@ -57,7 +58,6 @@
 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.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorDescriptor;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
@@ -419,16 +419,16 @@
                     + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
             ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount()
                     + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+            IDataFormat dataFormat = metadataProvider.getDataFormat();
 
             // add the previous record first
             int f = 0;
-            outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
+            outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(itemType);
             f++;
             // add the previous meta second
             if (dataset.hasMetaPart()) {
-                outputSerDes[f] =
-                        FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
-                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
+                outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(metaItemType);
+                outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(metaItemType);
                 f++;
             }
             // add the previous filter third
@@ -443,10 +443,10 @@
                     }
                 }
                 fieldIdx = i;
-                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider()
-                        .getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
-                outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider()
-                        .getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
+                outputTypeTraits[f] =
+                        dataFormat.getTypeTraitProvider().getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
+                outputSerDes[f] =
+                        dataFormat.getSerdeProvider().getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
                 f++;
             }
             for (int j = 0; j < inputRecordDesc.getFieldCount(); j++) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 445fbd8..78d9c19 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -212,10 +212,10 @@
         ITypeTraits[] enforcedTypeTraits =
                 new ITypeTraits[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
         IBinaryComparatorFactoryProvider comparatorFactoryProvider =
-                metadataProvider.getFormat().getBinaryComparatorFactoryProvider();
+                metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
         // Record column is 0 for external datasets, numPrimaryKeys for internal ones
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
@@ -230,9 +230,9 @@
                 sourceType = metaType;
                 sourceColumn = recordColumn + 1;
             }
-            secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : sourceType, index.getKeyFieldNames().get(i),
-                    sourceColumn);
+            secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
+                    index.getKeyFieldNames().get(i), sourceColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -270,8 +270,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 96fdf27..8ef5f34 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -137,10 +137,10 @@
         ITypeTraits[] enforcedTypeTraits =
                 new ITypeTraits[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
         IBinaryComparatorFactoryProvider comparatorFactoryProvider =
-                metadataProvider.getFormat().getBinaryComparatorFactoryProvider();
+                metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
         // Record column is 0 for external datasets, numPrimaryKeys for internal ones
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
         boolean isOverridingKeyTypes = index.isOverridingKeyFieldTypes();
@@ -155,9 +155,9 @@
                 sourceType = metaType;
                 sourceColumn = recordColumn + 1;
             }
-            secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyTypes ? enforcedItemType : sourceType, index.getKeyFieldNames().get(i),
-                    sourceColumn);
+            secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
+                    index.getKeyFieldNames().get(i), sourceColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -185,8 +185,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, recordColumn);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index 15f8a23..262b259 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -29,7 +29,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.om.utils.RecordUtil;
-import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -108,13 +107,13 @@
         ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
         if (numSecondaryKeys > 0) {
-            secondaryFieldAccessEvalFactories[0] = FormatUtils.getDefaultFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : itemType, index.getKeyFieldNames().get(0),
-                    recordColumn);
+            secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                    index.getKeyFieldNames().get(0), recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -124,8 +123,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = FormatUtils.getDefaultFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, recordColumn);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 8fd8a7a..b663b42 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -89,9 +89,9 @@
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
-        secondaryFieldAccessEvalFactories =
-                metadataProvider.getFormat().createMBRFactory(isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                        secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+        secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
+                metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 171d72e..b99ae2f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -32,7 +32,6 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexBulkLoadOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
@@ -155,10 +154,6 @@
     @Override
     protected AlgebricksMetaOperatorDescriptor createCastOp(JobSpecification spec, DatasetType dsType,
             boolean strictCast) throws AlgebricksException {
-        IFunctionDescriptor castFuncDesc = FunctionManagerHolder.getFunctionManager()
-                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
-        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
-
         int[] outColumns = new int[1];
 
         // tags(2) + primary keys + record + meta part(?)
@@ -185,7 +180,7 @@
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = castFuncDesc.createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { getTaggedRecordDescriptor(enforcedRecDesc) });
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 98c47a2..5dac407 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -22,7 +22,6 @@
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
 
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -52,7 +51,6 @@
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.IsUnknownDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -239,7 +237,7 @@
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
         primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
         primaryBloomFilterKeyFields = new int[numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
         List<Integer> indicators = null;
         if (dataset.hasMetaPart()) {
             indicators = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
@@ -298,10 +296,6 @@
 
     protected AlgebricksMetaOperatorDescriptor createCastOp(JobSpecification spec, DatasetType dsType,
             boolean strictCast) throws AlgebricksException {
-        IFunctionDescriptor castFuncDesc = FunctionManagerHolder.getFunctionManager()
-                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
-        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
-
         int[] outColumns = new int[1];
         int[] projectionList = new int[(dataset.hasMetaPart() ? 2 : 1) + numPrimaryKeys];
         int recordIdx;
@@ -322,12 +316,19 @@
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = castFuncDesc.createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { enforcedRecDesc });
     }
 
+    protected IFunctionDescriptor createCastFunction(boolean strictCast) throws AlgebricksException {
+        IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
+                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
+        castFuncDesc.setImmutableStates(enforcedItemType, itemType);
+        return castFuncDesc;
+    }
+
     protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
             IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
         int[] sortFields = new int[secondaryComparatorFactories.length];
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index b4d8a22..077e431 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -30,7 +30,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.om.utils.RecordUtil;
-import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -111,12 +110,12 @@
         ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
-        ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
         if (numSecondaryKeys > 0) {
-            secondaryFieldAccessEvalFactories[0] = FormatUtils.getDefaultFormat().getFieldAccessEvaluatorFactory(
-                    isOverridingKeyFieldTypes ? enforcedItemType : itemType, index.getKeyFieldNames().get(0),
-                    numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                    index.getKeyFieldNames().get(0), numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -126,8 +125,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] = FormatUtils.getDefaultFormat()
-                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 08ee0aa..1726470 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -99,9 +99,9 @@
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
-        secondaryFieldAccessEvalFactories =
-                metadataProvider.getFormat().createMBRFactory(isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                        secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+        secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
+                metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
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 7624a8a..678a864 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
@@ -20,16 +20,13 @@
 
 import java.util.List;
 
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -69,18 +66,16 @@
     public IMissingWriterFactory getMissingWriterFactory();
 
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            ARecordType recType, List<String> fldName) throws AlgebricksException;
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException;
 
-    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName,
-            int recordColumn) throws AlgebricksException;
-
-    public IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment typeEnvironment)
-            throws AlgebricksException;
+    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn) throws AlgebricksException;
 
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException;
 
-    public IScalarEvaluatorFactory[] createMBRFactory(ARecordType recType, List<String> fldName, int recordColumn,
-            int dimension, List<String> filterFieldName, boolean isPointMBR) throws AlgebricksException;
+    public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
+            throws AlgebricksException;
 
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
index 526eab6..0e746f9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
@@ -26,5 +26,13 @@
      *
      * @return a new IFunctionDescriptor instance
      */
-    public IFunctionDescriptor createFunctionDescriptor();
+    IFunctionDescriptor createFunctionDescriptor();
+
+    /**
+     * Creates type inferer for this function
+     * @return an new instance of the type inferer or {@code null} if not provided
+     */
+    default IFunctionTypeInferer createFunctionTypeInferer() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
new file mode 100644
index 0000000..fb8155a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.functions;
+
+/**
+ * An extension point for built-in functions
+ */
+public interface IFunctionExtensionManager {
+    /**
+     * Returns function manager
+     *
+     * @return a new {@link IFunctionManager} instance
+     */
+    IFunctionManager getFunctionManager();
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
index f7c11ff..e1657ff 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
@@ -21,11 +21,13 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
-public interface IFunctionManager extends Iterable<IFunctionDescriptorFactory> {
+/**
+ * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
+ * for built-in functions
+ */
+public interface IFunctionManager {
 
-    public void registerFunction(IFunctionDescriptorFactory descriptorFactory);
+    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
 
-    public void unregisterFunction(IFunctionDescriptorFactory descriptorFactory);
-
-    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
new file mode 100644
index 0000000..b8e27e1
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.functions;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public interface IFunctionTypeInferer {
+    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+            throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index f117d14..4e35950 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -21,22 +21,29 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class ListifyAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ListifyAggregateDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AOrderedListType oltype;
 
     @Override
@@ -53,5 +60,4 @@
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new ListifyAggregateFunctionEvalFactory(args, oltype);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index 96a298f..643098f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -18,15 +18,11 @@
  */
 package org.apache.asterix.runtime.aggregates.scalar;
 
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor.ScanCollectionUnnestingFunctionFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -37,6 +33,12 @@
 public abstract class AbstractScalarAggregateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
+    private final AbstractAggregateFunctionDynamicDescriptor aggFuncDesc;
+
+    protected AbstractScalarAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        this.aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) aggFuncDesc;
+    }
+
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
             throws AlgebricksException {
@@ -45,10 +47,6 @@
         IScalarEvaluatorFactory[] aggFuncArgs = new IScalarEvaluatorFactory[1];
         aggFuncArgs[0] = new ColumnAccessEvalFactory(0);
         // Create aggregate function from this scalar version.
-        FunctionIdentifier fid = BuiltinFunctions.getAggregateFunction(getIdentifier());
-        IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
-        IFunctionDescriptor fd = mgr.lookupFunction(fid);
-        AbstractAggregateFunctionDynamicDescriptor aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) fd;
         final IAggregateEvaluatorFactory aggFuncFactory = aggFuncDesc.createAggregateEvaluatorFactory(aggFuncArgs);
 
         return new IScalarEvaluatorFactory() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
index 51419f1..9fb994a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarAvgAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_AVG;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarAvgAggregateDescriptor();
+            return new ScalarAvgAggregateDescriptor(AvgAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarAvgAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
index 5bdb8b1..6aba99f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarCountAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_COUNT;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarCountAggregateDescriptor();
+            return new ScalarCountAggregateDescriptor(CountAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarCountAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
index f411bea..f18fd32 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarMaxAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_MAX;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarMaxAggregateDescriptor();
+            return new ScalarMaxAggregateDescriptor(MaxAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarMaxAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
index b1f95b0..17d008c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.MinAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarMinAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_MIN;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarMinAggregateDescriptor();
+            return new ScalarMinAggregateDescriptor(MinAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarMinAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
index 82c301c..ecc10d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlAvgAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlAvgAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_AVG;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlAvgAggregateDescriptor();
+            return new ScalarSqlAvgAggregateDescriptor(SqlAvgAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlAvgAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
index baea906..029956a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
@@ -21,18 +21,27 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlCountAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_COUNT;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlCountAggregateDescriptor();
+            return new ScalarSqlCountAggregateDescriptor(
+                    SqlCountAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlCountAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
index a182777..fb616b4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlMaxAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_MAX;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlMaxAggregateDescriptor();
+            return new ScalarSqlMaxAggregateDescriptor(SqlMaxAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    public ScalarSqlMaxAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
index 85da97c..123b955 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlMinAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlMinAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_MIN;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlMinAggregateDescriptor();
+            return new ScalarSqlMinAggregateDescriptor(SqlMinAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlMinAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
index fae1a1a..078ea57 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSqlSumAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_SUM;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSqlSumAggregateDescriptor();
+            return new ScalarSqlSumAggregateDescriptor(SqlSumAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSqlSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
index d534193..4d547e1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
@@ -21,18 +21,26 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class ScalarSumAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_SUM;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new ScalarSumAggregateDescriptor();
+            return new ScalarSumAggregateDescriptor(SumAggregateDescriptor.FACTORY.createFunctionDescriptor());
         }
     };
 
+    private ScalarSumAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
index 5cac892..f75f975 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
@@ -21,22 +21,29 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class ClosedRecordConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ClosedRecordConstructorDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
         }
     };
 
     private static final long serialVersionUID = 1L;
-
     private ARecordType recType;
 
     @Override
@@ -53,5 +60,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new ClosedRecordConstructorEvalFactory(args, recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
index 0b26583..c2c1b3f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
@@ -25,8 +25,10 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -44,6 +46,11 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OpenRecordConstructorDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.OpenRecordConstructorTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
index 977101a..3a378de 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
@@ -25,9 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -40,14 +42,19 @@
 
 public class OrderedListConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OrderedListConstructorDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AOrderedListType oltype;
 
     @Override
@@ -69,9 +76,7 @@
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory[] args;
-
         private boolean selfDescList = false;
-
         private AOrderedListType orderedlistType;
 
         public OrderedListConstructorEvaluatorFactory(IScalarEvaluatorFactory[] args, AOrderedListType type) {
@@ -142,6 +147,5 @@
             };
 
         }
-
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
index 50b7699..5f11acb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
@@ -25,10 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -41,14 +42,19 @@
 
 public class UnorderedListConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new UnorderedListConstructorDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_EXPRESSION_TYPE;
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private AUnorderedListType ultype;
 
     @Override
@@ -70,7 +76,6 @@
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory[] args;
-
         private boolean selfDescList = false;
         private boolean homoList = false;
         private AUnorderedListType unorderedlistType;
@@ -144,5 +149,4 @@
 
         }
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
index 7f5c58d..efc5f8a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
@@ -21,9 +21,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -37,10 +39,14 @@
 public class CastTypeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CastTypeDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.CastTypeInferer();
         }
     };
 
@@ -84,4 +90,3 @@
         };
     }
 }
-
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
index eea1b6e..51ea579 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
@@ -22,9 +22,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -50,6 +52,11 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CastTypeLaxDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.CastTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
index 150c28e..51497ed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
@@ -26,12 +26,14 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,10 +45,16 @@
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class DeepEqualityDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new DeepEqualityDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.DeepEqualityTypeInferer();
         }
     };
 
@@ -59,7 +67,6 @@
         this.inputTypeLeft = (IAType) states[0];
         this.inputTypeRight = (IAType) states[1];
     }
-
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
index 7ce7dec..d641ea9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class FieldAccessByIndexDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessByIndexDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FieldAccessByIndexTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new FieldAccessByIndexEvalFactory(args[0], args[1], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
index 95b06f7..5dd3e19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
@@ -23,20 +23,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class FieldAccessNestedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessNestedDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FieldAccessNestedTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
     private List<String> fldName;
 
@@ -55,5 +63,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new FieldAccessNestedEvalFactory(args[0], recType, fldName);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
index 3419b68..631dd70 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class GetRecordFieldValueDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GetRecordFieldValueDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.GetRecordFieldValueTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new GetRecordFieldValueEvalFactory(args[0], args[1], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
index 6205d77..273e3ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
@@ -21,20 +21,28 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class GetRecordFieldsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GetRecordFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.GetRecordFieldsTypeInferer();
+        }
     };
 
+    private static final long serialVersionUID = 1L;
     private ARecordType recType;
 
     @Override
@@ -51,5 +59,4 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new GetRecordFieldsEvalFactory(args[0], recType);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
index 5df04f8..97a9ee0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
@@ -31,6 +31,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.PointableAllocator;
@@ -47,6 +48,7 @@
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -67,7 +69,13 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordAddFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordAddFieldsTypeInferer();
+        }
     };
+
     private static final long serialVersionUID = 1L;
     private ARecordType outRecType;
     private ARecordType inRecType;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index 0b432ac..ce7ed6e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -42,6 +43,7 @@
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -68,7 +70,13 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordMergeDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordMergeTypeInferer();
+        }
     };
+
     private static final long serialVersionUID = 1L;
     private ARecordType outRecType;
     private ARecordType inRecType0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index 7158d2d..e5ed628 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -37,6 +38,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -54,6 +56,11 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordPairsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordPairsTypeInferer();
+        }
     };
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
index 092fbfb..909c74b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
@@ -21,28 +21,35 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 
 public class RecordRemoveFieldsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordRemoveFieldsDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.RecordRemoveFieldsTypeInferer();
+        }
     };
 
-    private RecordRemoveFieldsDescriptor() {
-    }
-
+    private static final long serialVersionUID = 1L;
     private ARecordType outputRecordType;
     private ARecordType inputRecType;
     private AOrderedListType inputListType;
+
+    private RecordRemoveFieldsDescriptor() {
+    }
 
     @Override
     public void setImmutableStates(Object... states) {
@@ -60,5 +67,4 @@
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.REMOVE_FIELDS;
     }
-
 }
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 090352d..e237319 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
@@ -19,11 +19,8 @@
 package org.apache.asterix.runtime.formats;
 
 import java.io.DataOutput;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -51,20 +48,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionManager;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
-import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
-import org.apache.asterix.runtime.functions.FunctionManagerHolder;
-import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
@@ -73,10 +61,8 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableEvalSizeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -94,35 +80,16 @@
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 
 public class NonTaggedDataFormat implements IDataFormat {
 
-    public static final NonTaggedDataFormat INSTANCE = new NonTaggedDataFormat();
+    static final NonTaggedDataFormat INSTANCE = new NonTaggedDataFormat();
 
     private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-    private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<>();
-
     public static final String NON_TAGGED_DATA_FORMAT = "org.apache.asterix.runtime.formats.NonTaggedDataFormat";
 
-    private final Map<FunctionIdentifier, FunctionTypeInferer> functionTypeInferers = new HashMap<>();
-
-    static {
-        typeToValueParserFactMap.put(ATypeTag.INTEGER, IntegerParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.BIGINT, LongParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-    }
-
-    public NonTaggedDataFormat() {
-        registerTypeInferers();
+    private NonTaggedDataFormat() {
     }
 
     @Override
@@ -152,8 +119,8 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName,
-            int recordColumn) throws AlgebricksException {
+    public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn) throws AlgebricksException {
         IScalarEvaluatorFactory recordEvalFactory = new ColumnAccessEvalFactory(recordColumn);
 
         if (fldName.size() == 1) {
@@ -173,8 +140,7 @@
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                            .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
                     fDesc.setImmutableStates(recType);
                     return fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -190,16 +156,14 @@
                 }
                 IScalarEvaluatorFactory fldNameEvalFactory =
                         new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                        .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
                 return fDesc.createEvaluatorFactory(
                         new IScalarEvaluatorFactory[] { recordEvalFactory, fldNameEvalFactory });
             }
         }
 
         if (fldName.size() > 1) {
-            IFunctionDescriptor fDesc =
-                    FunctionManagerHolder.getFunctionManager().lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
             fDesc.setImmutableStates(recType, fldName);
             return fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
         }
@@ -209,9 +173,11 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public IScalarEvaluatorFactory[] createMBRFactory(ARecordType recType, List<String> fldName, int recordColumn,
-            int dimension, List<String> filterFieldName, boolean isPointMBR) throws AlgebricksException {
-        IScalarEvaluatorFactory evalFactory = getFieldAccessEvaluatorFactory(recType, fldName, recordColumn);
+    public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
+            throws AlgebricksException {
+        IScalarEvaluatorFactory evalFactory =
+                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn);
         int numOfFields = isPointMBR ? dimension : dimension * 2;
         IScalarEvaluatorFactory[] evalFactories =
                 new IScalarEvaluatorFactory[numOfFields + (filterFieldName == null ? 0 : 1)];
@@ -242,7 +208,8 @@
             evalFactories[i] = new CreateMBREvalFactory(evalFactory, dimensionEvalFactory, coordinateEvalFactory);
         }
         if (filterFieldName != null) {
-            evalFactories[numOfFields] = getFieldAccessEvaluatorFactory(recType, filterFieldName, recordColumn);
+            evalFactories[numOfFields] =
+                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn);
         }
         return evalFactories;
     }
@@ -250,7 +217,7 @@
     @SuppressWarnings("unchecked")
     @Override
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            ARecordType recType, List<String> fldName) throws AlgebricksException {
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException {
         String[] names = recType.getFieldNames();
         int n = names.length;
         if (fldName.size() > 1) {
@@ -269,8 +236,7 @@
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = FunctionManagerHolder.getFunctionManager()
-                            .lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
                     fDesc.setImmutableStates(recType);
                     IScalarEvaluatorFactory evalFactory = fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -278,11 +244,9 @@
                             BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
 
                     ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                            new MutableObject<ILogicalExpression>(
-                                    new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
-                    return new Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory,
-                            partitionFun, recType.getFieldTypes()[i]);
+                            new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                            new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    return new Triple<>(evalFactory, partitionFun, recType.getFieldTypes()[i]);
                 }
             }
         } else {
@@ -296,286 +260,18 @@
             } catch (HyracksDataException e) {
                 throw new AlgebricksException(e);
             }
-            IFunctionDescriptor fDesc =
-                    FunctionManagerHolder.getFunctionManager().lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
             fDesc.setImmutableStates(recType, fldName);
             IScalarEvaluatorFactory evalFactory =
                     fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
             IFunctionInfo finfoAccess = BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED);
 
             ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(as))));
-            return new Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory, partitionFun,
-                    recType.getSubFieldType(fldName));
+                    new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                    new MutableObject<>(new ConstantExpression(new AsterixConstantValue(as))));
+            return new Triple<>(evalFactory, partitionFun, recType.getSubFieldType(fldName));
         }
         throw new AlgebricksException("Could not find field " + fldName + " in the schema.");
-    }
-
-    @Override
-    public IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment context)
-            throws AlgebricksException {
-        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
-        IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
-        IFunctionDescriptor fd = mgr.lookupFunction(fnId);
-        if (fd == null) {
-            throw new AlgebricksException("Unresolved function " + fnId);
-        }
-        final FunctionIdentifier fid = fd.getIdentifier();
-        if (functionTypeInferers.containsKey(fid)) {
-            functionTypeInferers.get(fid).infer(expr, fd, context);
-        }
-        return fd;
-    }
-
-    interface FunctionTypeInferer {
-        void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException;
-    }
-
-    void registerTypeInferers() {
-        functionTypeInferers.put(BuiltinFunctions.LISTIFY, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.RECORD_MERGE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.DEEP_EQUAL, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
-                fd.setImmutableStates(type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.ADD_FIELDS, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                ILogicalExpression listExpr = f.getArguments().get(1).getValue();
-                IAType type1 = (IAType) context.getType(listExpr);
-                if (type0.getTypeTag().equals(ATypeTag.ANY)) {
-                    type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                }
-                if (type1.getTypeTag().equals(ATypeTag.ANY)) {
-                    type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                }
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-
-        functionTypeInferers.put(BuiltinFunctions.REMOVE_FIELDS, new FunctionTypeInferer() {
-
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
-                IAType outType = (IAType) context.getType(expr);
-                IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
-                ILogicalExpression le = f.getArguments().get(1).getValue();
-                IAType type1 = (IAType) context.getType(le);
-                if (type0.getTypeTag().equals(ATypeTag.ANY)) {
-                    type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                }
-                if (type1.getTypeTag().equals(ATypeTag.ANY)) {
-                    type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                }
-                fd.setImmutableStates(outType, type0, type1);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CAST_TYPE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                IAType rt = TypeCastUtils.getRequiredType(funcExpr);
-                IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
-                fd.setImmutableStates(rt, it);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CAST_TYPE_LAX, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                IAType rt = TypeCastUtils.getRequiredType(funcExpr);
-                IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
-                fd.setImmutableStates(rt, it);
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                ARecordType rt = (ARecordType) context.getType(expr);
-                fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
-            }
-
-            private boolean[] computeOpenFields(AbstractFunctionCallExpression expr, ARecordType recType) {
-                int n = expr.getArguments().size() / 2;
-                boolean[] open = new boolean[n];
-                for (int i = 0; i < n; i++) {
-                    Mutable<ILogicalExpression> argRef = expr.getArguments().get(2 * i);
-                    ILogicalExpression arg = argRef.getValue();
-                    open[i] = true;
-                    final String fn = ConstantExpressionUtil.getStringConstant(arg);
-                    if (fn != null) {
-                        for (String s : recType.getFieldNames()) {
-                            if (s.equals(fn)) {
-                                open[i] = false;
-                                break;
-                            }
-                        }
-                    }
-                }
-                return open;
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                fd.setImmutableStates(context.getType(expr));
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                switch (t.getTypeTag()) {
-                    case OBJECT: {
-                        fd.setImmutableStates(t);
-                        break;
-                    }
-                    case UNION: {
-                        AUnionType unionT = (AUnionType) t;
-                        if (unionT.isUnknownableType()) {
-                            IAType t2 = unionT.getActualType();
-                            if (t2.getTypeTag() == ATypeTag.OBJECT) {
-                                fd.setImmutableStates(t2);
-                                break;
-                            }
-                        }
-                        throw new NotImplementedException("field-access-by-index for data of type " + t);
-                    }
-                    default: {
-                        throw new NotImplementedException("field-access-by-index for data of type " + t);
-                    }
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.FIELD_ACCESS_NESTED, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                AOrderedList fieldPath = (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce
-                        .getArguments().get(1).getValue()).getValue()).getObject());
-                List<String> listFieldPath = new ArrayList<String>();
-                for (int i = 0; i < fieldPath.size(); i++) {
-                    listFieldPath.add(((AString) fieldPath.getItem(i)).getStringValue());
-                }
-
-                switch (t.getTypeTag()) {
-                    case OBJECT: {
-                        fd.setImmutableStates(t, listFieldPath);
-                        break;
-                    }
-                    case ANY:
-                        fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
-                        break;
-                    default: {
-                        throw new NotImplementedException("field-access-nested for data of type " + t);
-                    }
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.GET_RECORD_FIELDS, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("get-record-fields for data of type " + t);
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.GET_RECORD_FIELD_VALUE, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("get-record-field-value for data of type " + t);
-                }
-            }
-        });
-        functionTypeInferers.put(BuiltinFunctions.RECORD_PAIRS, new FunctionTypeInferer() {
-            @Override
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                    throws AlgebricksException {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                ATypeTag typeTag = t.getTypeTag();
-                if (typeTag.equals(ATypeTag.OBJECT)) {
-                    fd.setImmutableStates(t);
-                } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-                } else {
-                    throw new NotImplementedException("record-fields with data of type " + t);
-                }
-            }
-        });
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index c02732f..5acbeb9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -331,394 +331,400 @@
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
 
 /**
- * This class (statically) holds a list of function descriptor factories.
+ * This class holds a list of function descriptor factories.
  */
 public final class FunctionCollection {
 
     private static final String FACTORY = "FACTORY";
-    private static final List<IFunctionDescriptorFactory> temp = new ArrayList<>();
 
-    static {
+    private final List<IFunctionDescriptorFactory> descriptorFactories = new ArrayList<>();
+
+    public void add(IFunctionDescriptorFactory descriptorFactory) {
+        descriptorFactories.add(descriptorFactory);
+    }
+
+    public void addGenerated(IFunctionDescriptorFactory descriptorFactory) {
+        add(getGeneratedFunctionDescriptorFactory(descriptorFactory.createFunctionDescriptor().getClass()));
+    }
+
+    public static FunctionCollection createDefaultFunctionCollection() {
+        FunctionCollection fc = new FunctionCollection();
+
         // unnesting function
-        temp.add(TidRunningAggregateDescriptor.FACTORY);
-        temp.add(ScanCollectionDescriptor.FACTORY);
-        temp.add(RangeDescriptor.FACTORY);
-        temp.add(SubsetCollectionDescriptor.FACTORY);
+        fc.add(TidRunningAggregateDescriptor.FACTORY);
+        fc.add(ScanCollectionDescriptor.FACTORY);
+        fc.add(RangeDescriptor.FACTORY);
+        fc.add(SubsetCollectionDescriptor.FACTORY);
 
         // aggregate functions
-        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);
-        temp.add(FirstElementAggregateDescriptor.FACTORY);
-        temp.add(LocalFirstElementAggregateDescriptor.FACTORY);
+        fc.add(ListifyAggregateDescriptor.FACTORY);
+        fc.add(CountAggregateDescriptor.FACTORY);
+        fc.add(AvgAggregateDescriptor.FACTORY);
+        fc.add(LocalAvgAggregateDescriptor.FACTORY);
+        fc.add(IntermediateAvgAggregateDescriptor.FACTORY);
+        fc.add(GlobalAvgAggregateDescriptor.FACTORY);
+        fc.add(SumAggregateDescriptor.FACTORY);
+        fc.add(LocalSumAggregateDescriptor.FACTORY);
+        fc.add(MaxAggregateDescriptor.FACTORY);
+        fc.add(LocalMaxAggregateDescriptor.FACTORY);
+        fc.add(MinAggregateDescriptor.FACTORY);
+        fc.add(LocalMinAggregateDescriptor.FACTORY);
+        fc.add(FirstElementAggregateDescriptor.FACTORY);
+        fc.add(LocalFirstElementAggregateDescriptor.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);
+        fc.add(SerializableCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableLocalAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableSumAggregateDescriptor.FACTORY);
+        fc.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);
-        temp.add(EmptyStreamAggregateDescriptor.FACTORY);
-        temp.add(NonEmptyStreamAggregateDescriptor.FACTORY);
+        fc.add(ScalarCountAggregateDescriptor.FACTORY);
+        fc.add(ScalarAvgAggregateDescriptor.FACTORY);
+        fc.add(ScalarSumAggregateDescriptor.FACTORY);
+        fc.add(ScalarMaxAggregateDescriptor.FACTORY);
+        fc.add(ScalarMinAggregateDescriptor.FACTORY);
+        fc.add(EmptyStreamAggregateDescriptor.FACTORY);
+        fc.add(NonEmptyStreamAggregateDescriptor.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);
+        fc.add(SqlCountAggregateDescriptor.FACTORY);
+        fc.add(SqlAvgAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(IntermediateSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(GlobalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SqlSumAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlSumAggregateDescriptor.FACTORY);
+        fc.add(SqlMaxAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlMaxAggregateDescriptor.FACTORY);
+        fc.add(SqlMinAggregateDescriptor.FACTORY);
+        fc.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);
+        fc.add(SerializableSqlCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableLocalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableIntermediateSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableGlobalSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(SerializableSqlSumAggregateDescriptor.FACTORY);
+        fc.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);
+        fc.add(ScalarSqlCountAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlAvgAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlSumAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlMaxAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlMinAggregateDescriptor.FACTORY);
 
         // boolean functions
-        temp.add(AndDescriptor.FACTORY);
-        temp.add(OrDescriptor.FACTORY);
+        fc.add(AndDescriptor.FACTORY);
+        fc.add(OrDescriptor.FACTORY);
 
         // Record constructors
-        temp.add(ClosedRecordConstructorDescriptor.FACTORY);
-        temp.add(OpenRecordConstructorDescriptor.FACTORY);
+        fc.add(ClosedRecordConstructorDescriptor.FACTORY);
+        fc.add(OpenRecordConstructorDescriptor.FACTORY);
 
         // List constructors
-        temp.add(OrderedListConstructorDescriptor.FACTORY);
-        temp.add(UnorderedListConstructorDescriptor.FACTORY);
+        fc.add(OrderedListConstructorDescriptor.FACTORY);
+        fc.add(UnorderedListConstructorDescriptor.FACTORY);
 
         // Sleep function
-        temp.add(SleepDescriptor.FACTORY);
+        fc.add(SleepDescriptor.FACTORY);
 
         // Inject failure function
-        temp.add(InjectFailureDescriptor.FACTORY);
+        fc.add(InjectFailureDescriptor.FACTORY);
 
         // Switch case
-        temp.add(SwitchCaseDescriptor.FACTORY);
+        fc.add(SwitchCaseDescriptor.FACTORY);
 
         // null functions
-        temp.add(IsMissingDescriptor.FACTORY);
-        temp.add(IsNullDescriptor.FACTORY);
-        temp.add(IsUnknownDescriptor.FACTORY);
-        temp.add(IsSystemNullDescriptor.FACTORY);
-        temp.add(CheckUnknownDescriptor.FACTORY);
-        temp.add(IfMissingDescriptor.FACTORY);
-        temp.add(IfNullDescriptor.FACTORY);
-        temp.add(IfMissingOrNullDescriptor.FACTORY);
+        fc.add(IsMissingDescriptor.FACTORY);
+        fc.add(IsNullDescriptor.FACTORY);
+        fc.add(IsUnknownDescriptor.FACTORY);
+        fc.add(IsSystemNullDescriptor.FACTORY);
+        fc.add(CheckUnknownDescriptor.FACTORY);
+        fc.add(IfMissingDescriptor.FACTORY);
+        fc.add(IfNullDescriptor.FACTORY);
+        fc.add(IfMissingOrNullDescriptor.FACTORY);
 
         // uuid generators (zero independent functions)
-        temp.add(CreateUUIDDescriptor.FACTORY);
-        temp.add(UUIDDescriptor.FACTORY);
-        temp.add(CreateQueryUIDDescriptor.FACTORY);
-        temp.add(CurrentDateDescriptor.FACTORY);
-        temp.add(CurrentTimeDescriptor.FACTORY);
-        temp.add(CurrentDateTimeDescriptor.FACTORY);
+        fc.add(CreateUUIDDescriptor.FACTORY);
+        fc.add(UUIDDescriptor.FACTORY);
+        fc.add(CreateQueryUIDDescriptor.FACTORY);
+        fc.add(CurrentDateDescriptor.FACTORY);
+        fc.add(CurrentTimeDescriptor.FACTORY);
+        fc.add(CurrentDateTimeDescriptor.FACTORY);
 
         // TODO: decide how should we deal these two weird functions as
         // the number of arguments of the function depend on the first few arguments.
-        temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
-        temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
+        fc.add(SimilarityJaccardPrefixDescriptor.FACTORY);
+        fc.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
 
         // functions that need generated class for null-handling.
-        List<IFunctionDescriptorFactory> functionsToInjectUnkownHandling = new ArrayList<>();
 
         // Element accessors.
-        functionsToInjectUnkownHandling.add(FieldAccessByIndexDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FieldAccessByNameDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FieldAccessNestedDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AnyCollectionMemberDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetItemDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessByIndexDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessByNameDescriptor.FACTORY);
+        fc.addGenerated(FieldAccessNestedDescriptor.FACTORY);
+
+        fc.addGenerated(AnyCollectionMemberDescriptor.FACTORY);
+        fc.addGenerated(GetItemDescriptor.FACTORY);
 
         // Numeric functions
-        functionsToInjectUnkownHandling.add(NumericUnaryMinusDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericAddDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericDivideDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericMultiplyDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSubDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericModuloDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCaretDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NotDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LenDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericAbsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCeilingDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericFloorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundHalfToEvenDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericRoundHalfToEven2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericACosDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericASinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericATanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericCosDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericTanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericExpDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericLnDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericLogDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSqrtDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericSignDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericTruncDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NumericATan2Descriptor.FACTORY);
+        fc.addGenerated(NumericUnaryMinusDescriptor.FACTORY);
+        fc.addGenerated(NumericAddDescriptor.FACTORY);
+        fc.addGenerated(NumericDivideDescriptor.FACTORY);
+        fc.addGenerated(NumericMultiplyDescriptor.FACTORY);
+        fc.addGenerated(NumericSubDescriptor.FACTORY);
+        fc.addGenerated(NumericModuloDescriptor.FACTORY);
+        fc.addGenerated(NumericCaretDescriptor.FACTORY);
+        fc.addGenerated(NotDescriptor.FACTORY);
+        fc.addGenerated(LenDescriptor.FACTORY);
+        fc.addGenerated(NumericAbsDescriptor.FACTORY);
+        fc.addGenerated(NumericCeilingDescriptor.FACTORY);
+        fc.addGenerated(NumericFloorDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundHalfToEvenDescriptor.FACTORY);
+        fc.addGenerated(NumericRoundHalfToEven2Descriptor.FACTORY);
+        fc.addGenerated(NumericACosDescriptor.FACTORY);
+        fc.addGenerated(NumericASinDescriptor.FACTORY);
+        fc.addGenerated(NumericATanDescriptor.FACTORY);
+        fc.addGenerated(NumericCosDescriptor.FACTORY);
+        fc.addGenerated(NumericSinDescriptor.FACTORY);
+        fc.addGenerated(NumericTanDescriptor.FACTORY);
+        fc.addGenerated(NumericExpDescriptor.FACTORY);
+        fc.addGenerated(NumericLnDescriptor.FACTORY);
+        fc.addGenerated(NumericLogDescriptor.FACTORY);
+        fc.addGenerated(NumericSqrtDescriptor.FACTORY);
+        fc.addGenerated(NumericSignDescriptor.FACTORY);
+        fc.addGenerated(NumericTruncDescriptor.FACTORY);
+        fc.addGenerated(NumericATan2Descriptor.FACTORY);
 
         // Comparisons.
-        functionsToInjectUnkownHandling.add(EqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GreaterThanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GreaterThanOrEqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LessThanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(LessThanOrEqualsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(NotEqualsDescriptor.FACTORY);
+        fc.addGenerated(EqualsDescriptor.FACTORY);
+        fc.addGenerated(GreaterThanDescriptor.FACTORY);
+        fc.addGenerated(GreaterThanOrEqualsDescriptor.FACTORY);
+        fc.addGenerated(LessThanDescriptor.FACTORY);
+        fc.addGenerated(LessThanOrEqualsDescriptor.FACTORY);
+        fc.addGenerated(NotEqualsDescriptor.FACTORY);
 
         // Binary functions
-        functionsToInjectUnkownHandling.add(BinaryLengthDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(BinaryConcatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubBinaryFromDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubBinaryFromToDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FindBinaryDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FindBinaryFromDescriptor.FACTORY);
+        fc.addGenerated(BinaryLengthDescriptor.FACTORY);
+        fc.addGenerated(ParseBinaryDescriptor.FACTORY);
+        fc.addGenerated(PrintBinaryDescriptor.FACTORY);
+        fc.addGenerated(BinaryConcatDescriptor.FACTORY);
+        fc.addGenerated(SubBinaryFromDescriptor.FACTORY);
+        fc.addGenerated(SubBinaryFromToDescriptor.FACTORY);
+        fc.addGenerated(FindBinaryDescriptor.FACTORY);
+        fc.addGenerated(FindBinaryFromDescriptor.FACTORY);
 
         // String functions
-        functionsToInjectUnkownHandling.add(StringLikeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringEndsWithDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringStartsWithDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringEqualDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLowerCaseDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringUpperCaseDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLengthDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(Substring2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringBeforeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SubstringAfterDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringToCodePointDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CodePointToStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringConcatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringJoinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpContainsWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpLikeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpLikeWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpPositionDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpPositionWithFlagDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpReplaceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRegExpReplaceWithFlagsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringInitCapDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRTrimDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringLTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRTrim2Descriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringPositionDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringRepeatDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(StringSplitDescriptor.FACTORY);
+        fc.addGenerated(StringLikeDescriptor.FACTORY);
+        fc.addGenerated(StringContainsDescriptor.FACTORY);
+        fc.addGenerated(StringEndsWithDescriptor.FACTORY);
+        fc.addGenerated(StringStartsWithDescriptor.FACTORY);
+        fc.addGenerated(SubstringDescriptor.FACTORY);
+        fc.addGenerated(StringEqualDescriptor.FACTORY);
+        fc.addGenerated(StringLowerCaseDescriptor.FACTORY);
+        fc.addGenerated(StringUpperCaseDescriptor.FACTORY);
+        fc.addGenerated(StringLengthDescriptor.FACTORY);
+        fc.addGenerated(Substring2Descriptor.FACTORY);
+        fc.addGenerated(SubstringBeforeDescriptor.FACTORY);
+        fc.addGenerated(SubstringAfterDescriptor.FACTORY);
+        fc.addGenerated(StringToCodePointDescriptor.FACTORY);
+        fc.addGenerated(CodePointToStringDescriptor.FACTORY);
+        fc.addGenerated(StringConcatDescriptor.FACTORY);
+        fc.addGenerated(StringJoinDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpContainsDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpContainsWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpLikeDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpLikeWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpPositionDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpPositionWithFlagDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpReplaceDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpReplaceWithFlagsDescriptor.FACTORY);
+        fc.addGenerated(StringInitCapDescriptor.FACTORY);
+        fc.addGenerated(StringTrimDescriptor.FACTORY);
+        fc.addGenerated(StringLTrimDescriptor.FACTORY);
+        fc.addGenerated(StringRTrimDescriptor.FACTORY);
+        fc.addGenerated(StringTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringLTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringRTrim2Descriptor.FACTORY);
+        fc.addGenerated(StringPositionDescriptor.FACTORY);
+        fc.addGenerated(StringRepeatDescriptor.FACTORY);
+        fc.addGenerated(StringSplitDescriptor.FACTORY);
 
         // Constructors
-        functionsToInjectUnkownHandling.add(ABooleanConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ABinaryHexStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ABinaryBase64StringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt8ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt16ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt32ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AInt64ConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AFloatConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADoubleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APointConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APoint3DConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ALineConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(APolygonConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ACircleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ARectangleConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ATimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADateConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADateTimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AYearMonthDurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ADayTimeDurationConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AUUIDFromStringConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromDateConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABooleanConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABinaryHexStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(ABinaryBase64StringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt8ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt16ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt32ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AInt64ConstructorDescriptor.FACTORY);
+        fc.addGenerated(AFloatConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADoubleConstructorDescriptor.FACTORY);
+        fc.addGenerated(APointConstructorDescriptor.FACTORY);
+        fc.addGenerated(APoint3DConstructorDescriptor.FACTORY);
+        fc.addGenerated(ALineConstructorDescriptor.FACTORY);
+        fc.addGenerated(APolygonConstructorDescriptor.FACTORY);
+        fc.addGenerated(ACircleConstructorDescriptor.FACTORY);
+        fc.addGenerated(ARectangleConstructorDescriptor.FACTORY);
+        fc.addGenerated(ATimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADateConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADateTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(AYearMonthDurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(ADayTimeDurationConstructorDescriptor.FACTORY);
+        fc.addGenerated(AUUIDFromStringConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromDateConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
+        fc.addGenerated(AIntervalStartFromTimeConstructorDescriptor.FACTORY);
 
         // Spatial
-        functionsToInjectUnkownHandling.add(CreatePointDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateLineDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreatePolygonDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateCircleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateRectangleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialAreaDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialDistanceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialIntersectDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CreateMBRDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SpatialCellDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PointXCoordinateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(PointYCoordinateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(CircleRadiusAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(CircleCenterAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(LineRectanglePolygonAccessor.FACTORY);
+        fc.addGenerated(CreatePointDescriptor.FACTORY);
+        fc.addGenerated(CreateLineDescriptor.FACTORY);
+        fc.addGenerated(CreatePolygonDescriptor.FACTORY);
+        fc.addGenerated(CreateCircleDescriptor.FACTORY);
+        fc.addGenerated(CreateRectangleDescriptor.FACTORY);
+        fc.addGenerated(SpatialAreaDescriptor.FACTORY);
+        fc.addGenerated(SpatialDistanceDescriptor.FACTORY);
+        fc.addGenerated(SpatialIntersectDescriptor.FACTORY);
+        fc.addGenerated(CreateMBRDescriptor.FACTORY);
+        fc.addGenerated(SpatialCellDescriptor.FACTORY);
+        fc.addGenerated(PointXCoordinateAccessor.FACTORY);
+        fc.addGenerated(PointYCoordinateAccessor.FACTORY);
+        fc.addGenerated(CircleRadiusAccessor.FACTORY);
+        fc.addGenerated(CircleCenterAccessor.FACTORY);
+        fc.addGenerated(LineRectanglePolygonAccessor.FACTORY);
 
         // fuzzyjoin function
-        functionsToInjectUnkownHandling.add(PrefixLenJaccardDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(WordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(HashedWordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CountHashedWordTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(HashedGramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CountHashedGramTokensDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceCheckDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceStringIsFilterableDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceListIsFilterableDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(EditDistanceContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardCheckDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardSortedDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(SimilarityJaccardSortedCheckDescriptor.FACTORY);
+        fc.addGenerated(PrefixLenJaccardDescriptor.FACTORY);
+        fc.addGenerated(WordTokensDescriptor.FACTORY);
+        fc.addGenerated(HashedWordTokensDescriptor.FACTORY);
+        fc.addGenerated(CountHashedWordTokensDescriptor.FACTORY);
+        fc.addGenerated(GramTokensDescriptor.FACTORY);
+        fc.addGenerated(HashedGramTokensDescriptor.FACTORY);
+        fc.addGenerated(CountHashedGramTokensDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceCheckDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceStringIsFilterableDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceListIsFilterableDescriptor.FACTORY);
+        fc.addGenerated(EditDistanceContainsDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardCheckDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardSortedDescriptor.FACTORY);
+        fc.addGenerated(SimilarityJaccardSortedCheckDescriptor.FACTORY);
 
         // full-text function
-        functionsToInjectUnkownHandling.add(FullTextContainsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(FullTextContainsWithoutOptionDescriptor.FACTORY);
+        fc.addGenerated(FullTextContainsDescriptor.FACTORY);
+        fc.addGenerated(FullTextContainsWithoutOptionDescriptor.FACTORY);
 
         // Record functions.
-        functionsToInjectUnkownHandling.add(GetRecordFieldsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetRecordFieldValueDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DeepEqualityDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordMergeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordAddFieldsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(RecordRemoveFieldsDescriptor.FACTORY);
+        fc.addGenerated(GetRecordFieldsDescriptor.FACTORY);
+        fc.addGenerated(GetRecordFieldValueDescriptor.FACTORY);
+        fc.addGenerated(DeepEqualityDescriptor.FACTORY);
+        fc.addGenerated(RecordMergeDescriptor.FACTORY);
+        fc.addGenerated(RecordAddFieldsDescriptor.FACTORY);
+        fc.addGenerated(RecordRemoveFieldsDescriptor.FACTORY);
 
         // Spatial and temporal type accessors
-        functionsToInjectUnkownHandling.add(TemporalYearAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMonthAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalDayAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalHourAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMinuteAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalSecondAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalMillisecondAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartDateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndDateAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartTimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndTimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalStartDatetimeAccessor.FACTORY);
-        functionsToInjectUnkownHandling.add(TemporalIntervalEndDatetimeAccessor.FACTORY);
+        fc.addGenerated(TemporalYearAccessor.FACTORY);
+        fc.addGenerated(TemporalMonthAccessor.FACTORY);
+        fc.addGenerated(TemporalDayAccessor.FACTORY);
+        fc.addGenerated(TemporalHourAccessor.FACTORY);
+        fc.addGenerated(TemporalMinuteAccessor.FACTORY);
+        fc.addGenerated(TemporalSecondAccessor.FACTORY);
+        fc.addGenerated(TemporalMillisecondAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartDateAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndDateAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartTimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndTimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalStartDatetimeAccessor.FACTORY);
+        fc.addGenerated(TemporalIntervalEndDatetimeAccessor.FACTORY);
 
         // Temporal functions
-        functionsToInjectUnkownHandling.add(UnixTimeFromDateInDaysDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromDatetimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(UnixTimeFromDatetimeInSecsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DateFromDatetimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(TimeFromDatetimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CalendarDuartionFromDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(AdjustTimeForTimeZoneDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalBeforeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalAfterDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalMeetsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalMetByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalOverlapsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalOverlappedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(OverlapDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalStartsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalStartedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalCoversDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalCoveredByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalEndsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalEndedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromMillisecondsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromMonthsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(YearMonthDurationGreaterThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(YearMonthDurationLessThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayTimeDurationGreaterThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayTimeDurationLessThanComparatorDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(MonthsFromYearMonthDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationEqualDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetYearMonthDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetDayTimeDurationDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IntervalBinDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(OverlapBinsDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DayOfWeekDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ParseDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintDateDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(PrintDateTimeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(GetOverlappingIntervalDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(DurationFromIntervalDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDateInDaysDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDatetimeInMsDescriptor.FACTORY);
+        fc.addGenerated(UnixTimeFromDatetimeInSecsDescriptor.FACTORY);
+        fc.addGenerated(DateFromUnixTimeInDaysDescriptor.FACTORY);
+        fc.addGenerated(DateFromDatetimeDescriptor.FACTORY);
+        fc.addGenerated(TimeFromUnixTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(TimeFromDatetimeDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
+        fc.addGenerated(DatetimeFromDateAndTimeDescriptor.FACTORY);
+        fc.addGenerated(CalendarDurationFromDateTimeDescriptor.FACTORY);
+        fc.addGenerated(CalendarDuartionFromDateDescriptor.FACTORY);
+        fc.addGenerated(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
+        fc.addGenerated(AdjustTimeForTimeZoneDescriptor.FACTORY);
+        fc.addGenerated(IntervalBeforeDescriptor.FACTORY);
+        fc.addGenerated(IntervalAfterDescriptor.FACTORY);
+        fc.addGenerated(IntervalMeetsDescriptor.FACTORY);
+        fc.addGenerated(IntervalMetByDescriptor.FACTORY);
+        fc.addGenerated(IntervalOverlapsDescriptor.FACTORY);
+        fc.addGenerated(IntervalOverlappedByDescriptor.FACTORY);
+        fc.addGenerated(OverlapDescriptor.FACTORY);
+        fc.addGenerated(IntervalStartsDescriptor.FACTORY);
+        fc.addGenerated(IntervalStartedByDescriptor.FACTORY);
+        fc.addGenerated(IntervalCoversDescriptor.FACTORY);
+        fc.addGenerated(IntervalCoveredByDescriptor.FACTORY);
+        fc.addGenerated(IntervalEndsDescriptor.FACTORY);
+        fc.addGenerated(IntervalEndedByDescriptor.FACTORY);
+        fc.addGenerated(DurationFromMillisecondsDescriptor.FACTORY);
+        fc.addGenerated(DurationFromMonthsDescriptor.FACTORY);
+        fc.addGenerated(YearMonthDurationGreaterThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(YearMonthDurationLessThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(DayTimeDurationGreaterThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(DayTimeDurationLessThanComparatorDescriptor.FACTORY);
+        fc.addGenerated(MonthsFromYearMonthDurationDescriptor.FACTORY);
+        fc.addGenerated(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
+        fc.addGenerated(DurationEqualDescriptor.FACTORY);
+        fc.addGenerated(GetYearMonthDurationDescriptor.FACTORY);
+        fc.addGenerated(GetDayTimeDurationDescriptor.FACTORY);
+        fc.addGenerated(IntervalBinDescriptor.FACTORY);
+        fc.addGenerated(OverlapBinsDescriptor.FACTORY);
+        fc.addGenerated(DayOfWeekDescriptor.FACTORY);
+        fc.addGenerated(ParseDateDescriptor.FACTORY);
+        fc.addGenerated(ParseTimeDescriptor.FACTORY);
+        fc.addGenerated(ParseDateTimeDescriptor.FACTORY);
+        fc.addGenerated(PrintDateDescriptor.FACTORY);
+        fc.addGenerated(PrintTimeDescriptor.FACTORY);
+        fc.addGenerated(PrintDateTimeDescriptor.FACTORY);
+        fc.addGenerated(GetOverlappingIntervalDescriptor.FACTORY);
+        fc.addGenerated(DurationFromIntervalDescriptor.FACTORY);
 
         // Type functions.
-        functionsToInjectUnkownHandling.add(IsBooleanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsNumberDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsArrayDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(IsObjectDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToBooleanDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToStringDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToDoubleDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(ToBigIntDescriptor.FACTORY);
+        fc.addGenerated(IsBooleanDescriptor.FACTORY);
+        fc.addGenerated(IsNumberDescriptor.FACTORY);
+        fc.addGenerated(IsStringDescriptor.FACTORY);
+        fc.addGenerated(IsArrayDescriptor.FACTORY);
+        fc.addGenerated(IsObjectDescriptor.FACTORY);
+        fc.addGenerated(ToBooleanDescriptor.FACTORY);
+        fc.addGenerated(ToStringDescriptor.FACTORY);
+        fc.addGenerated(ToDoubleDescriptor.FACTORY);
+        fc.addGenerated(ToBigIntDescriptor.FACTORY);
 
         // Cast function
-        functionsToInjectUnkownHandling.add(CastTypeDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(CastTypeLaxDescriptor.FACTORY);
+        fc.addGenerated(CastTypeDescriptor.FACTORY);
+        fc.addGenerated(CastTypeLaxDescriptor.FACTORY);
 
         // Record function
-        functionsToInjectUnkownHandling.add(RecordPairsDescriptor.FACTORY);
+        fc.addGenerated(RecordPairsDescriptor.FACTORY);
 
-        List<IFunctionDescriptorFactory> generatedFactories = new ArrayList<>();
-        for (IFunctionDescriptorFactory factory : functionsToInjectUnkownHandling) {
-            generatedFactories
-                    .add(getGeneratedFunctionDescriptorFactory(factory.createFunctionDescriptor().getClass()));
-        }
-        temp.addAll(generatedFactories);
+        return fc;
     }
 
-    public static List<IFunctionDescriptorFactory> getFunctionDescriptorFactories() {
-        return temp;
+    public List<IFunctionDescriptorFactory> getFunctionDescriptorFactories() {
+        return descriptorFactories;
     }
 
     /**
@@ -739,8 +745,5 @@
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
-    }
-
-    private FunctionCollection() {
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
new file mode 100644
index 0000000..d5a6559
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
@@ -0,0 +1,73 @@
+/*
+ * 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.functions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * Default implementation of {@link IFunctionManager}.
+ */
+public final class FunctionManager implements IFunctionManager {
+
+    private final Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functions;
+
+    private final Map<FunctionIdentifier, IFunctionTypeInferer> typeInferers;
+
+    public FunctionManager(FunctionCollection functionCollection) {
+        Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functionsMap = new HashMap<>();
+        Map<FunctionIdentifier, IFunctionTypeInferer> typeInferersMap = new HashMap<>();
+
+        for (IFunctionDescriptorFactory descriptorFactory : functionCollection.getFunctionDescriptorFactories()) {
+            FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
+            functionsMap.put(new Pair<>(fid, fid.getArity()), descriptorFactory);
+            IFunctionTypeInferer typeInferer = descriptorFactory.createFunctionTypeInferer();
+            if (typeInferer != null) {
+                typeInferersMap.put(fid, typeInferer);
+            }
+        }
+
+        this.functions = functionsMap;
+        this.typeInferers = typeInferersMap;
+    }
+
+    @Override
+    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
+        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
+        IFunctionDescriptorFactory factory = functions.get(key);
+        if (factory == null) {
+            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
+        }
+        return factory.createFunctionDescriptor();
+    }
+
+    @Override
+    public IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid) {
+        return typeInferers.get(fid);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
deleted file mode 100644
index 52b61bb..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.functions;
-
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.functions.IFunctionManager;
-
-public final class FunctionManagerHolder {
-    private static final IFunctionManager functionManager = createFunctionManager();
-
-    public static IFunctionManager getFunctionManager() {
-        return functionManager;
-    }
-
-    private static IFunctionManager createFunctionManager() {
-        FunctionManagerImpl mgr = new FunctionManagerImpl();
-        for (IFunctionDescriptorFactory fdFactory : FunctionCollection.getFunctionDescriptorFactories()) {
-            mgr.registerFunction(fdFactory);
-        }
-        return mgr;
-    }
-
-    private FunctionManagerHolder() {
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
deleted file mode 100644
index 4056955..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.runtime.functions;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.functions.IFunctionManager;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-
-final class FunctionManagerImpl implements IFunctionManager {
-    private final Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functions;
-
-    FunctionManagerImpl() {
-        functions = new HashMap<>();
-    }
-
-    @Override
-    public synchronized IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
-        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
-        IFunctionDescriptorFactory factory = functions.get(key);
-        if (factory == null) {
-            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
-        }
-        return factory.createFunctionDescriptor();
-    }
-
-    @Override
-    public synchronized void registerFunction(IFunctionDescriptorFactory descriptorFactory) {
-        FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        functions.put(new Pair<>(fid, fid.getArity()), descriptorFactory);
-    }
-
-    @Override
-    public synchronized void unregisterFunction(IFunctionDescriptorFactory descriptorFactory) {
-        FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
-        functions.remove(key);
-    }
-
-    @Override
-    public synchronized Iterator<IFunctionDescriptorFactory> iterator() {
-        return new ArrayList<>(functions.values()).iterator();
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
new file mode 100644
index 0000000..00a5ec8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -0,0 +1,271 @@
+/*
+ * 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.functions;
+
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementations of {@link IFunctionTypeInferer} for built-in functions
+ */
+public final class FunctionTypeInferers {
+
+    private FunctionTypeInferers() {
+    }
+
+    public static final IFunctionTypeInferer SET_EXPRESSION_TYPE = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            fd.setImmutableStates(context.getType(expr));
+        }
+    };
+
+    public static final class CastTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            IAType rt = TypeCastUtils.getRequiredType(funcExpr);
+            IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+            fd.setImmutableStates(rt, it);
+        }
+    }
+
+    public static final class DeepEqualityTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
+            fd.setImmutableStates(type0, type1);
+        }
+    }
+
+    public static final class FieldAccessByIndexTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            switch (t.getTypeTag()) {
+                case OBJECT: {
+                    fd.setImmutableStates(t);
+                    break;
+                }
+                case UNION: {
+                    AUnionType unionT = (AUnionType) t;
+                    if (unionT.isUnknownableType()) {
+                        IAType t2 = unionT.getActualType();
+                        if (t2.getTypeTag() == ATypeTag.OBJECT) {
+                            fd.setImmutableStates(t2);
+                            break;
+                        }
+                    }
+                    throw new NotImplementedException("field-access-by-index for data of type " + t);
+                }
+                default: {
+                    throw new NotImplementedException("field-access-by-index for data of type " + t);
+                }
+            }
+        }
+    }
+
+    public static final class FieldAccessNestedTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            AOrderedList fieldPath =
+                    (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce.getArguments().get(1).getValue())
+                            .getValue()).getObject());
+            List<String> listFieldPath = new ArrayList<>();
+            for (int i = 0; i < fieldPath.size(); i++) {
+                listFieldPath.add(((AString) fieldPath.getItem(i)).getStringValue());
+            }
+
+            switch (t.getTypeTag()) {
+                case OBJECT: {
+                    fd.setImmutableStates(t, listFieldPath);
+                    break;
+                }
+                case ANY:
+                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
+                    break;
+                default: {
+                    throw new NotImplementedException("field-access-nested for data of type " + t);
+                }
+            }
+        }
+    }
+
+    public static final class GetRecordFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("get-record-fields for data of type " + t);
+            }
+        }
+    }
+
+    public static final class GetRecordFieldValueTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("get-record-field-value for data of type " + t);
+            }
+        }
+    }
+
+    public static final class OpenRecordConstructorTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            ARecordType rt = (ARecordType) context.getType(expr);
+            fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
+        }
+
+        private boolean[] computeOpenFields(AbstractFunctionCallExpression expr, ARecordType recType) {
+            int n = expr.getArguments().size() / 2;
+            boolean[] open = new boolean[n];
+            for (int i = 0; i < n; i++) {
+                Mutable<ILogicalExpression> argRef = expr.getArguments().get(2 * i);
+                ILogicalExpression arg = argRef.getValue();
+                open[i] = true;
+                final String fn = ConstantExpressionUtil.getStringConstant(arg);
+                if (fn != null) {
+                    for (String s : recType.getFieldNames()) {
+                        if (s.equals(fn)) {
+                            open[i] = false;
+                            break;
+                        }
+                    }
+                }
+            }
+            return open;
+        }
+    }
+
+    public static final class RecordAddFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            ILogicalExpression listExpr = f.getArguments().get(1).getValue();
+            IAType type1 = (IAType) context.getType(listExpr);
+            if (type0.getTypeTag().equals(ATypeTag.ANY)) {
+                type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            if (type1.getTypeTag().equals(ATypeTag.ANY)) {
+                type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+
+    public static final class RecordMergeTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+
+    public static final class RecordPairsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("record-fields with data of type " + t);
+            }
+        }
+    }
+
+    public static final class RecordRemoveFieldsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException {
+            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+            IAType outType = (IAType) context.getType(expr);
+            IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
+            ILogicalExpression le = f.getArguments().get(1).getValue();
+            IAType type1 = (IAType) context.getType(le);
+            if (type0.getTypeTag().equals(ATypeTag.ANY)) {
+                type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            if (type1.getTypeTag().equals(ATypeTag.ANY)) {
+                type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            fd.setImmutableStates(outType, type0, type1);
+        }
+    }
+}

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2118

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................

[ASTERIXDB-2154][COMP] Extensible built-in function management

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

Details:
- Support for augmenting built-in functions in ILangExtension
- Move function type inferers from NonTaggedDataFormat to
  FunctionTypeInferers, exposed through IFunctionDescriptorFactory
  and IFunctionManager
- Cleanup: obtain IFunctionManager/IDataFormat from MetadataProvider

Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionExtensionManager.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlAvgAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMaxAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMinAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ClosedRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OrderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/UnorderedListConstructorDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerHolder.java
D asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManagerImpl.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
59 files changed, 1,159 insertions(+), 927 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/18/2118/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/2271/ (6/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Ian Maxon (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ian Maxon has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/1420/ (7/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2154][COMP] Extensible built-in function management

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2154][COMP] Extensible built-in function management
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6961/ (3/10)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2118
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iafbf7e4e298fb7af6cb84b402cb00cfa2e90b5ce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No