You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/04/05 21:40:24 UTC

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905 )


Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................

[ASTERIXDB-2871][COMP] Improve UDF recursion detection

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

Details:
- Improve UDF recursion detection
- Rewrite UDF body expression only once
- Add testcases

Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
---
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/translator/QueryTranslator.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
D asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
A asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
46 files changed, 1,393 insertions(+), 840 deletions(-)



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

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 0091304..2af2815 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
@@ -184,8 +184,9 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
+            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output,
+            boolean allowNonStoredUdfCalls, boolean inlineUdfs, Collection<VarIdentifier> externalVars,
+            IWarningCollector warningCollector) throws CompilationException {
         if (q == null) {
             return null;
         }
@@ -194,8 +195,9 @@
             generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        rw.rewrite(new ArrayList<>(declaredFunctions), q, metadataProvider,
-                new LangRewritingContext(q.getVarCounter(), warningCollector), inlineUdfs, externalVars);
+        LangRewritingContext rwCtx =
+                new LangRewritingContext(metadataProvider, declaredFunctions, warningCollector, q.getVarCounter());
+        rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfs, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index d8698f9..60a96d4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -95,13 +95,17 @@
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
+import org.apache.asterix.lang.common.literal.MissingLiteral;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
@@ -2158,16 +2162,30 @@
                     }
                 }
 
-                //Check whether the function is use-able
+                // Check whether the function is usable:
+                // create a function declaration for this function,
+                // and a query body calls this function with each argument set to 'missing'
+                FunctionDecl fd = new FunctionDecl(functionSignature, paramVars, cfs.getFunctionBodyExpression(), true);
+                fd.setSourceLocation(sourceLoc);
+                CallExpr fcall = new CallExpr(functionSignature,
+                        Collections.nCopies(paramVars.size(), new LiteralExpr(MissingLiteral.INSTANCE)));
+                fcall.setSourceLocation(sourceLoc);
                 metadataProvider.setDefaultDataverse(dv);
                 Query wrappedQuery = new Query(false);
                 wrappedQuery.setSourceLocation(sourceLoc);
-                wrappedQuery.setBody(cfs.getFunctionBodyExpression());
+                wrappedQuery.setBody(fcall);
                 wrappedQuery.setTopLevel(false);
-                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false,
-                        paramVars, warningCollector);
-                List<List<Triple<DataverseName, String, String>>> dependencies = FunctionUtil.getFunctionDependencies(
-                        rewriterFactory.createQueryRewriter(), cfs.getFunctionBodyExpression());
+                List<FunctionDecl> fdList = new ArrayList<>(declaredFunctions);
+                fdList.add(fd);
+                apiFramework.reWriteQuery(fdList, metadataProvider, wrappedQuery, sessionOutput, false, false,
+                        Collections.emptyList(), warningCollector);
+                Expression fdNormBody = fd.getNormalizedFuncBody();
+                if (fdNormBody == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                            functionSignature.toString());
+                }
+                List<List<Triple<DataverseName, String, String>>> dependencies =
+                        FunctionUtil.getFunctionDependencies(rewriterFactory.createQueryRewriter(), fdNormBody);
 
                 newInlineTypes = Collections.emptyMap();
                 function = new Function(functionSignature, paramNames, null, null, cfs.getFunctionBody(),
@@ -2864,7 +2882,7 @@
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
         Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, query, sessionOutput, true, externalVars.keySet(), warningCollector);
+                metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -2881,7 +2899,7 @@
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
         Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, insertUpsert, sessionOutput, true, externalVars.keySet(), warningCollector);
+                metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         DataverseName dataverseName = getActiveDataverseName(rewrittenInsertUpsert.getDataverseName());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8b6f05f..98f328d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -27,15 +27,17 @@
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintWriter;
+import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.IParser;
@@ -187,8 +189,9 @@
                 if (st.getKind() == Statement.Kind.QUERY) {
                     Query query = (Query) st;
                     IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
-                    rewrite(rewriter, functions, query, metadataProvider,
-                            new LangRewritingContext(query.getVarCounter(), TestUtils.NOOP_WARNING_COLLECTOR));
+                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions,
+                            TestUtils.NOOP_WARNING_COLLECTOR, query.getVarCounter());
+                    rewrite(rewriter, query, rwContext);
 
                     // Tests deep copy and deep equality.
                     Query copiedQuery = (Query) SqlppRewriteUtil.deepCopy(query);
@@ -254,28 +257,38 @@
     }
 
     // Rewrite queries.
-    // Note: we do not do inline function rewriting here because this needs real
-    // metadata access.
-    private void rewrite(IQueryRewriter rewriter, List<FunctionDecl> declaredFunctions, Query topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException {
-        PA.invokeMethod(rewriter,
-                "setup(java.util.List, org.apache.asterix.lang.common.base.IReturningStatement, "
-                        + "org.apache.asterix.metadata.declared.MetadataProvider, "
-                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext, " + "java.util.Collection)",
-                declaredFunctions, topExpr, metadataProvider, context, null);
-        PA.invokeMethod(rewriter, "resolveFunctionCalls()");
-        PA.invokeMethod(rewriter, "generateColumnNames()");
-        PA.invokeMethod(rewriter, "substituteGroupbyKeyExpression()");
-        PA.invokeMethod(rewriter, "rewriteGroupBys()");
-        PA.invokeMethod(rewriter, "rewriteSetOperations()");
-        PA.invokeMethod(rewriter, "inlineColumnAlias()");
-        PA.invokeMethod(rewriter, "rewriteWindowExpressions()");
-        PA.invokeMethod(rewriter, "rewriteGroupingSets()");
-        PA.invokeMethod(rewriter, "variableCheckAndRewrite()");
-        PA.invokeMethod(rewriter, "extractAggregatesFromCaseExpressions()");
-        PA.invokeMethod(rewriter, "rewriteGroupByAggregationSugar()");
-        PA.invokeMethod(rewriter, "rewriteWindowAggregationSugar()");
-        PA.invokeMethod(rewriter, "rewriteSpecialFunctionNames()");
+    // Note: we do not do inline function rewriting here because this needs real metadata access.
+    private void rewrite(IQueryRewriter rewriter, Query topExpr, LangRewritingContext context) throws Exception {
+        invokeMethod(rewriter, "setup", context, topExpr, null, true, false);
+        invokeMethod(rewriter, "resolveFunctionCalls");
+        invokeMethod(rewriter, "generateColumnNames");
+        invokeMethod(rewriter, "substituteGroupbyKeyExpression");
+        invokeMethod(rewriter, "rewriteGroupBys");
+        invokeMethod(rewriter, "rewriteSetOperations");
+        invokeMethod(rewriter, "inlineColumnAlias");
+        invokeMethod(rewriter, "rewriteWindowExpressions");
+        invokeMethod(rewriter, "rewriteGroupingSets");
+        invokeMethod(rewriter, "variableCheckAndRewrite");
+        invokeMethod(rewriter, "extractAggregatesFromCaseExpressions");
+        invokeMethod(rewriter, "rewriteGroupByAggregationSugar");
+        invokeMethod(rewriter, "rewriteWindowAggregationSugar");
+        invokeMethod(rewriter, "rewriteSpecialFunctionNames");
+        invokeMethod(rewriter, "rewriteOperatorExpression");
+        invokeMethod(rewriter, "rewriteCaseExpressions");
+        invokeMethod(rewriter, "rewriteListInputFunctions");
+        invokeMethod(rewriter, "rewriteRightJoins");
     }
 
+    private static void invokeMethod(Object instance, String methodName, Object... args) throws Exception {
+        PA.invokeMethod(instance, getMethodSignature(instance.getClass(), methodName), args);
+    }
+
+    private static String getMethodSignature(Class<?> cls, String methodName) throws Exception {
+        Method[] methods = cls.getDeclaredMethods();
+        Method method = Arrays.stream(methods).filter(m -> m.getName().equals(methodName)).findFirst()
+                .orElseThrow(NoSuchMethodException::new);
+        String parameterTypes =
+                Arrays.stream(method.getParameterTypes()).map(Class::getName).collect(Collectors.joining(","));
+        return String.format("%s(%s)", method.getName(), parameterTypes);
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
index 58fe98b..2e1d5a8 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
@@ -1,17 +1,15 @@
 Query:
 SELECT ELEMENT [
-CASE    LiteralExpr [TRUE]
-
-WHEN     OperatorExpr [
-      Variable [ Name=$i ]
-      >
-      Variable [ Name=$j ]
-    ]
-THEN     Variable [ Name=$i ]
-
-ELSE     Variable [ Name=$j ]
-
-END
+FunctionCall asterix.switch-case[
+  LiteralExpr [TRUE]
+  OperatorExpr [
+    Variable [ Name=$i ]
+    >
+    Variable [ Name=$j ]
+  ]
+  Variable [ Name=$i ]
+  Variable [ Name=$j ]
+]
 ]
 FROM [  OrderedListConstructor [
     LiteralExpr [LONG] [1]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
index cf95780..eeed914 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
@@ -1,15 +1,9 @@
 Query:
 SELECT ELEMENT [
-CASE    OperatorExpr [
-      LiteralExpr [LONG] [2]
-      >
-      LiteralExpr [LONG] [1]
-    ]
-
-WHEN     LiteralExpr [TRUE]
-THEN     LiteralExpr [LONG] [20]
-
-ELSE     LiteralExpr [LONG] [10]
-
-END
+FunctionCall asterix.switch-case[
+  LiteralExpr [TRUE]
+  LiteralExpr [TRUE]
+  LiteralExpr [LONG] [20]
+  LiteralExpr [LONG] [10]
+]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
index 42831c4..5fb51ac 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
@@ -4,18 +4,12 @@
   SELECT ELEMENT [
   Variable [ Name=$k ]
   ]
-  FROM [    CASE        OperatorExpr [
-          Variable [ Name=$i ]
-          >
-          Variable [ Name=$j ]
-        ]
-
-    WHEN         LiteralExpr [TRUE]
-    THEN         Variable [ Name=$i ]
-
-    ELSE         Variable [ Name=$j ]
-
-    END
+  FROM [    FunctionCall asterix.switch-case[
+      LiteralExpr [TRUE]
+      LiteralExpr [TRUE]
+      Variable [ Name=$i ]
+      Variable [ Name=$j ]
+    ]
     AS Variable [ Name=$k ]
   ]
   Where
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
index 3434965..d40210f 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
@@ -4,18 +4,16 @@
   SELECT ELEMENT [
   Variable [ Name=$k ]
   ]
-  FROM [    CASE        LiteralExpr [TRUE]
-
-    WHEN         OperatorExpr [
-          Variable [ Name=$i ]
-          >
-          Variable [ Name=$j ]
-        ]
-    THEN         Variable [ Name=$i ]
-
-    ELSE         Variable [ Name=$j ]
-
-    END
+  FROM [    FunctionCall asterix.switch-case[
+      LiteralExpr [TRUE]
+      OperatorExpr [
+        Variable [ Name=$i ]
+        >
+        Variable [ Name=$j ]
+      ]
+      Variable [ Name=$i ]
+      Variable [ Name=$j ]
+    ]
     AS Variable [ Name=$k ]
   ]
   Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
new file mode 100644
index 0000000..cf1603e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test illegal recursive function calls
+ */
+drop dataverse test if exists;
+create dataverse test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
new file mode 100644
index 0000000..ee465fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f2a(a) { - f2a(a) };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
new file mode 100644
index 0000000..79fc6bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f3a(a) { -a };
+create or replace function f3a(a) { f3a(a) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
new file mode 100644
index 0000000..24af0d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f4a(a) { -a };
+create function f4b(b) { f4a(b) };
+create or replace function f4a(a) { f4b(a) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
new file mode 100644
index 0000000..c6ded74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f5a(a) { - f5a(a) };
+
+f5a(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
new file mode 100644
index 0000000..f47b781
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f6a(a) { f6b(a) + f6b(-a) };
+declare function f6b(b) { f6a(-b) + f6a(b) };
+
+f6a(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
new file mode 100644
index 0000000..0c89f14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f7a(a) { f7b(a) + f7b(-a) };
+declare function f7b(b) { f7a(-b) + f7a(b) };
+declare function f7c(c) { f7b(c) };
+
+f7c(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
new file mode 100644
index 0000000..bf5ca06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION (varargs)
+ * Expected Res : Failure
+ */
+
+use test;
+
+create function f8a(...) { - args[0] - args[1] };
+
+create function f8b(...) { f8a(args[0], args[1]) + f8a(args[1], args[0]) };
+
+create or replace function f8a(...) { f8b(args[0], args[1]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
new file mode 100644
index 0000000..e5d4e98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION (varargs)
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f9a(...) { f9b(args[0]) + f9b(-args[1]) };
+
+declare function f9b(...) { f9a(-args[0]) + f9a(args[1]) };
+
+declare function f9c(...) { f9b(args[0], args[1]) };
+
+f9c(1, 2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
new file mode 100644
index 0000000..f237df5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function f1a(a) { -a };
+
+create function f1b(b) { f1a(b) + f1a(b) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
new file mode 100644
index 0000000..2c4236b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f1a(r) f1a, f1b(r) f1b
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab206d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+create function f3a(a) { -a };
+
+create function f3b(b) { f3a(b) };
+
+create function f3c(c) { f3a(c) + f3b(c) };
+
+create function f3d(d) { f3a(d) + f3b(d) + f3c(d) };
+
+create function f3e(e) { f3a(e) + f3b(e) + f3c(e) + f3d(e) };
+
+create function f3f(f) { f3a(f) + f3b(f) + f3c(f) + f3d(f) + f3e(f) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
new file mode 100644
index 0000000..2ad1f20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f3a(r) f3a, f3b(r) f3b, f3c(r) f3c, f3d(r) f3d, f3e(r) f3e, f3f(r) f3f
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
new file mode 100644
index 0000000..e4ce3f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
@@ -0,0 +1,37 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion (vararg functions)
+ */
+
+use test;
+
+create function f5a(...) { - args[0] - args[1] };
+
+create function f5b(...) { f5a(args[0], args[1]) };
+
+create function f5c(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) };
+
+create function f5d(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) };
+
+create function f5e(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) + f5d(args[0], args[1]) };
+
+create function f5f(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) + f5d(args[0], args[1]) + f5e(args[0], args[1]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
new file mode 100644
index 0000000..5e8087b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f5a(r, r+1) f5a, f5b(r, r+1) f5b, f5c(r, r+1) f5c, f5d(r, r+1) f5d, f5e(r, r+1) f5e, f5f(r, r+1) f5f
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
new file mode 100644
index 0000000..b7e7b81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description  : Cannot call a declared function from
+ *                a stored function
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+declare function f1a(a) { -a };
+
+create function f1b(b) { f1a(b) + f1a(b) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
new file mode 100644
index 0000000..0ebb577
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Cannot call a declared function from
+ *                a stored function (varargs)
+ */
+
+use test;
+
+declare function f2a(...) { - args[0] - args[1] };
+
+create function f2b(...) { f2a(args[0], args[1]) + f2a(args[1], args[0]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
new file mode 100644
index 0000000..c82ed63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f1a": -1, "f1b": -2 }
+{ "r": 2, "f1a": -2, "f1b": -4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
new file mode 100644
index 0000000..5c663b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f3a": -1, "f3b": -1, "f3c": -2, "f3d": -4, "f3e": -8, "f3f": -16 }
+{ "r": 2, "f3a": -2, "f3b": -2, "f3c": -4, "f3d": -8, "f3e": -16, "f3f": -32 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
new file mode 100644
index 0000000..6389c0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f5a": -3, "f5b": -3, "f5c": -6, "f5d": -12, "f5e": -24, "f5f": -48 }
+{ "r": 2, "f5a": -5, "f5b": -5, "f5c": -10, "f5d": -20, "f5e": -40, "f5f": -80 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 19e0965..04a274a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -12341,6 +12341,31 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf37_recursion">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1149: Illegal function recursion (in line 24, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 25, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 26, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 27, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 28, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 31, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf38_no_recursion">
+        <output-dir compare="Text">udf38_no_recursion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf39_illegal_call">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1150: Illegal use of function test.f1a(1) (in line 32, at column 26)</expected-error>
+        <expected-error>ASX1150: Illegal use of function test.f2a(...) (in line 29, at column 28)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="f01">
         <output-dir compare="Text">f01</output-dir>
         <expected-error>ASX1081: Cannot find function with signature test.tinyint()</expected-error>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 47a388d..210ce0f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -233,6 +233,8 @@
     UNKNOWN_FEED_POLICY(1146),
     CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS(1147),
     CANNOT_DROP_OBJECT_DEPENDENT_EXISTS(1148),
+    ILLEGAL_FUNCTION_RECURSION(1149),
+    ILLEGAL_FUNCTION_USE(1150),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 8954766..ab65a46 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -235,6 +235,8 @@
 1146 = Cannot find feed policy with name %1$s
 1147 = Cannot drop dataverse: %1$s %2$s being used by %3$s %4$s
 1148 = Cannot drop %1$s %2$s being used by %3$s %4$s
+1149 = Illegal function recursion
+1150 = Illegal use of function %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
index 0fe58ff..3df490c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
@@ -35,7 +35,8 @@
 
     List<String> parseMultipartIdentifier() throws CompilationException;
 
-    FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames) throws CompilationException;
+    FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
+            throws CompilationException;
 
     /**
      * Gets the warnings generated during parsing
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 3e1851c..f28de55 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -19,40 +19,37 @@
 package org.apache.asterix.lang.common.base;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 
 public interface IQueryRewriter {
 
     /**
      * Rewrite a query at the AST level.
-     * @param declaredFunctions,
-     *          a list of declared functions associated with the query.
      * @param topExpr,
      *          the query to be rewritten.
-     * @param metadataProvider,
-     *          providing the definition of created (i.e., stored) user-defined functions.
      * @param context
      *          rewriting context
+     * @param allowNonStoredUdfCalls
+     *          whether calls to non-stored user-defined functions should be resolved
+     * @param inlineUdfs
+     *          whether user defined functions should be inlines
      * @param externalVars
-     *          external variables
+     *          statement parameters (external variables)
      */
-    void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context, boolean inlineUdfs, Collection<VarIdentifier> externalVars)
-            throws CompilationException;
+    void rewrite(LangRewritingContext context, IReturningStatement topExpr, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException;
 
     /**
      * Find the function calls used by a given expression
      */
-    Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException;
+    void getFunctionCalls(Expression expression, Collection<? super AbstractCallExpression> outCalls)
+            throws CompilationException;
 
     /**
      * Find all external variables (positional and named variables) in given expression
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
deleted file mode 100644
index d18aa86..0000000
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
+++ /dev/null
@@ -1,62 +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.lang.common.parser;
-
-import java.io.StringReader;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.metadata.entities.Function;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-
-public class FunctionParser {
-
-    private final IParserFactory parserFactory;
-
-    public FunctionParser(IParserFactory parserFactory) {
-        this.parserFactory = parserFactory;
-    }
-
-    public String getLanguage() {
-        return parserFactory.getLanguage();
-    }
-
-    public FunctionDecl getFunctionDecl(Function function, IWarningCollector warningCollector)
-            throws CompilationException {
-        if (!function.getLanguage().equals(getLanguage())) {
-            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE, getLanguage(),
-                    function.getLanguage());
-        }
-        IParser parser = parserFactory.createParser(new StringReader(function.getFunctionBody()));
-        try {
-            FunctionDecl functionDecl = parser.parseFunctionBody(function.getSignature(), function.getParameterNames());
-            if (warningCollector != null) {
-                parser.getWarnings(warningCollector);
-            }
-            return functionDecl;
-        } catch (CompilationException e) {
-            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, function.getSignature(),
-                    e.getMessage());
-        }
-    }
-}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 563e07e..675d0d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -19,21 +19,31 @@
 package org.apache.asterix.lang.common.rewrites;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
 public final class LangRewritingContext {
+    private final MetadataProvider metadataProvider;
     private final IWarningCollector warningCollector;
-    private Counter varCounter;
+    private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
+    private final Counter varCounter;
     private int systemVarCounter = 1;
-    private Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
+    private final Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
 
-    public LangRewritingContext(int varCounter, IWarningCollector warningCollector) {
-        this.varCounter = new Counter(varCounter);
+    public LangRewritingContext(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions,
+            IWarningCollector warningCollector, int varCounter) {
+        this.metadataProvider = metadataProvider;
         this.warningCollector = warningCollector;
+        this.declaredFunctions = FunctionUtil.getFunctionMap(declaredFunctions);
+        this.varCounter = new Counter(varCounter);
     }
 
     public Counter getVarCounter() {
@@ -75,4 +85,12 @@
     public IWarningCollector getWarningCollector() {
         return warningCollector;
     }
+
+    public MetadataProvider getMetadataProvider() {
+        return metadataProvider;
+    }
+
+    public Map<FunctionSignature, FunctionDecl> getDeclaredFunctions() {
+        return declaredFunctions;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
index 4161824..2ef11ad 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
@@ -29,14 +29,18 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class FunctionDecl extends AbstractStatement {
-    private FunctionSignature signature;
-    private List<VarIdentifier> paramList;
+    private final FunctionSignature signature;
+    private final List<VarIdentifier> paramList;
     private Expression funcBody;
+    private Expression funcBodyNormalized;
+    private final boolean isStored;
 
-    public FunctionDecl(FunctionSignature signature, List<VarIdentifier> paramList, Expression funcBody) {
+    public FunctionDecl(FunctionSignature signature, List<VarIdentifier> paramList, Expression funcBody,
+            boolean isStored) {
         this.signature = signature;
         this.paramList = paramList;
         this.funcBody = funcBody;
+        this.isStored = isStored;
     }
 
     public FunctionSignature getSignature() {
@@ -53,14 +57,19 @@
 
     public void setFuncBody(Expression funcBody) {
         this.funcBody = funcBody;
+        this.funcBodyNormalized = null;
     }
 
-    public void setSignature(FunctionSignature signature) {
-        this.signature = signature;
+    public Expression getNormalizedFuncBody() {
+        return funcBodyNormalized;
     }
 
-    public void setParamList(List<VarIdentifier> paramList) {
-        this.paramList = paramList;
+    public void setNormalizedFuncBody(Expression funcBody) {
+        this.funcBodyNormalized = funcBody;
+    }
+
+    public boolean isStored() {
+        return isStored;
     }
 
     @Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 7f4e078..3a6bf97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -19,11 +19,14 @@
 
 package org.apache.asterix.lang.common.util;
 
+import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.function.BiFunction;
@@ -34,6 +37,8 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -41,8 +46,8 @@
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
-import org.apache.asterix.lang.common.parser.FunctionParser;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.BuiltinTypeMap;
 import org.apache.asterix.metadata.entities.Dataverse;
@@ -62,6 +67,10 @@
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
+import com.google.common.graph.GraphBuilder;
+import com.google.common.graph.Graphs;
+import com.google.common.graph.MutableGraph;
+
 public class FunctionUtil {
 
     public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
@@ -104,54 +113,50 @@
         }
     }
 
-    @FunctionalInterface
-    public interface IFunctionCollector {
-        Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException;
-    }
-
     public static FunctionSignature resolveFunctionCall(FunctionSignature fs, SourceLocation sourceLoc,
-            MetadataProvider metadataProvider, Set<FunctionSignature> declaredFunctions,
-            BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver) throws CompilationException {
-        int arity = fs.getArity();
+            MetadataProvider metadataProvider, BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver,
+            boolean searchUdfs, Map<FunctionSignature, FunctionDecl> declaredFunctionMap,
+            boolean allowNonStoredUdfCalls) throws CompilationException {
         DataverseName dataverse = fs.getDataverseName();
         if (dataverse == null) {
             dataverse = metadataProvider.getDefaultDataverseName();
         }
-        boolean isBuiltinFuncDataverse =
-                dataverse.equals(FunctionConstants.ASTERIX_DV) || dataverse.equals(FunctionConstants.ALGEBRICKS_DV);
-
-        if (!isBuiltinFuncDataverse) {
+        if (searchUdfs && !isBuiltinFunctionDataverse(dataverse)) {
             // attempt to resolve to a user-defined function
             FunctionSignature fsWithDv =
-                    fs.getDataverseName() == null ? new FunctionSignature(dataverse, fs.getName(), arity) : fs;
-            if (declaredFunctions.contains(fsWithDv)) {
-                return fsWithDv;
-            }
+                    fs.getDataverseName() == null ? new FunctionSignature(dataverse, fs.getName(), fs.getArity()) : fs;
             FunctionSignature fsWithDvVarargs =
                     new FunctionSignature(fsWithDv.getDataverseName(), fsWithDv.getName(), FunctionIdentifier.VARARGS);
-            if (declaredFunctions.contains(fsWithDvVarargs)) {
-                return fsWithDvVarargs;
+
+            FunctionDecl fd = declaredFunctionMap.get(fsWithDv);
+            if (fd == null) {
+                fd = declaredFunctionMap.get(fsWithDvVarargs);
+            }
+            if (fd != null) {
+                if (!allowNonStoredUdfCalls && !fd.isStored()) {
+                    throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_USE, sourceLoc,
+                            fd.getSignature().toString());
+                }
+                return fd.getSignature();
             }
             try {
-                Function function = metadataProvider.lookupUserDefinedFunction(fsWithDv);
-                if (function != null) {
-                    return fsWithDv;
+                Function fn = metadataProvider.lookupUserDefinedFunction(fsWithDv);
+                if (fn == null) {
+                    fn = metadataProvider.lookupUserDefinedFunction(fsWithDvVarargs);
                 }
-                function = metadataProvider.lookupUserDefinedFunction(fsWithDvVarargs);
-                if (function != null) {
-                    return fsWithDvVarargs;
+                if (fn != null) {
+                    return fn.getSignature();
                 }
             } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
+                throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, e, sourceLoc, fs.toString());
             }
-
             // fail if the dataverse was specified in the function call but this dataverse does not exist
             if (fs.getDataverseName() != null) {
                 Dataverse dv;
                 try {
                     dv = metadataProvider.findDataverse(dataverse);
                 } catch (AlgebricksException e) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, dataverse);
                 }
                 if (dv == null) {
                     throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverse);
@@ -165,13 +170,22 @@
         if (mappedName != null) {
             name = mappedName;
         }
-        FunctionSignature fsBuiltin = builtinFunctionResolver.apply(name, arity);
+        FunctionSignature fsBuiltin = builtinFunctionResolver.apply(name, fs.getArity());
         if (fsBuiltin == null) {
             throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, fs.toString());
         }
         return fsBuiltin;
     }
 
+    public static boolean isBuiltinFunctionSignature(FunctionSignature fs) {
+        return isBuiltinFunctionDataverse(Objects.requireNonNull(fs.getDataverseName()))
+                || BuiltinFunctions.getBuiltinFunctionInfo(fs.createFunctionIdentifier()) != null;
+    }
+
+    private static boolean isBuiltinFunctionDataverse(DataverseName dataverse) {
+        return FunctionConstants.ASTERIX_DV.equals(dataverse) || FunctionConstants.ALGEBRICKS_DV.equals(dataverse);
+    }
+
     public static BiFunction<String, Integer, FunctionSignature> createBuiltinFunctionResolver(
             MetadataProvider metadataProvider) {
         boolean includePrivateFunctions = getImportPrivateFunctions(metadataProvider);
@@ -193,88 +207,38 @@
         };
     }
 
-    /**
-     * Retrieve stored functions (from CREATE FUNCTION statements) that have been
-     * used in an expression.
-     *
-     * @param metadataProvider,
-     *            the metadata provider
-     * @param expression,
-     *            the expression for analysis
-     * @param declaredFunctions,
-     *            a set of declared functions in the query, which can potentially
-     *            override stored functions.
-     * @param functionCollector,
-     *            for collecting function calls in the <code>expression</code>
-     * @param functionParser,
-     *            for parsing stored functions in the string represetnation.
-     * @param warningCollector
-     *            for reporting warnings encountered during parsing
-     * @throws CompilationException
-     */
-    public static List<FunctionDecl> retrieveUsedStoredFunctions(MetadataProvider metadataProvider,
-            Expression expression, List<FunctionSignature> declaredFunctions, List<FunctionDecl> inputFunctionDecls,
-            IFunctionCollector functionCollector, FunctionParser functionParser, IWarningCollector warningCollector)
-            throws CompilationException {
-        if (expression == null) {
-            return Collections.emptyList();
-        }
-        List<FunctionDecl> functionDecls =
-                inputFunctionDecls == null ? new ArrayList<>() : new ArrayList<>(inputFunctionDecls);
-        Set<AbstractCallExpression> functionCalls = functionCollector.getFunctionCalls(expression);
-        Set<FunctionSignature> functionSignatures = new HashSet<>();
-        for (AbstractCallExpression functionCall : functionCalls) {
-            switch (functionCall.getKind()) {
-                case CALL_EXPRESSION:
-                    FunctionSignature fs = functionCall.getFunctionSignature();
-                    if (fs.getDataverseName() == null) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
-                                functionCall.getSourceLocation(), fs);
+    public static void checkFunctionRecursion(Map<FunctionSignature, FunctionDecl> functionDeclMap,
+            java.util.function.Function<Collection<AbstractCallExpression>, GatherFunctionCallsVisitor> gfcFactory,
+            SourceLocation sourceLoc) throws CompilationException {
+        List<AbstractCallExpression> callList = new ArrayList<>();
+        GatherFunctionCallsVisitor gfc = gfcFactory.apply(callList);
+        MutableGraph<FunctionDecl> graph = GraphBuilder.directed().allowsSelfLoops(true).build();
+        for (FunctionDecl fdFrom : functionDeclMap.values()) {
+            callList.clear();
+            fdFrom.getNormalizedFuncBody().accept(gfc, null);
+            for (AbstractCallExpression callExpr : callList) {
+                if (callExpr.getKind() == Expression.Kind.CALL_EXPRESSION) {
+                    FunctionSignature callSignature = callExpr.getFunctionSignature();
+                    FunctionDecl fdTo = functionDeclMap.get(callSignature);
+                    if (fdTo != null) {
+                        graph.putEdge(fdFrom, fdTo);
                     }
-                    if (!functionSignatures.add(fs)) {
-                        // already seen this signature
-                        continue;
-                    }
-                    if (declaredFunctions != null && declaredFunctions.contains(fs)) {
-                        continue;
-                    }
-                    Function function;
-                    try {
-                        function = metadataProvider.lookupUserDefinedFunction(fs);
-                    } catch (AlgebricksException e) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, functionCall.getSourceLocation(),
-                                e.toString());
-                    }
-                    if (function == null || !functionParser.getLanguage().equals(function.getLanguage())) {
-                        // the function is either unknown, builtin, or in a different language.
-                        // either way we ignore it here because it will be handled by the function inlining rule later
-                        continue;
-                    }
-
-                    FunctionDecl functionDecl = functionParser.getFunctionDecl(function, warningCollector);
-                    if (functionDecls.contains(functionDecl)) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
-                                "Recursive invocation " + functionDecls.get(functionDecls.size() - 1).getSignature()
-                                        + " <==> " + functionDecl.getSignature());
-                    }
-                    functionDecls.add(functionDecl);
-                    functionDecls = retrieveUsedStoredFunctions(metadataProvider, functionDecl.getFuncBody(),
-                            declaredFunctions, functionDecls, functionCollector, functionParser, warningCollector);
-                    break;
-                case WINDOW_EXPRESSION:
-                    // there cannot be used-defined window functions
-                    break;
-                default:
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expression.getSourceLocation(),
-                            functionCall.getFunctionSignature().toString(false));
+                }
             }
         }
-        return functionDecls;
+        if (Graphs.hasCycle(graph)) {
+            throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_RECURSION, sourceLoc);
+        }
     }
 
     public static List<List<Triple<DataverseName, String, String>>> getFunctionDependencies(IQueryRewriter rewriter,
             Expression expression) throws CompilationException {
-        Set<AbstractCallExpression> functionCalls = rewriter.getFunctionCalls(expression);
+        List<AbstractCallExpression> functionCalls = new ArrayList<>();
+        rewriter.getFunctionCalls(expression, functionCalls);
+        // Duplicate elimination
+        Set<FunctionSignature> seenFunctions = new HashSet<>();
+        Set<Pair<DataverseName, String>> seenDatasets = new HashSet<>();
+        Set<Pair<DataverseName, String>> seenSynonyms = new HashSet<>();
         //Get the List of used functions and used datasets
         List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
         List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
@@ -289,17 +253,23 @@
                         if (callExpr.getExprList().size() > 2) {
                             // resolved via synonym -> store synonym name as a dependency
                             Pair<DataverseName, String> synonymReference = parseDatasetFunctionArguments(callExpr, 2);
-                            synonymDependencies
-                                    .add(new Triple<>(synonymReference.first, synonymReference.second, null));
+                            if (seenSynonyms.add(synonymReference)) {
+                                synonymDependencies
+                                        .add(new Triple<>(synonymReference.first, synonymReference.second, null));
+                            }
                         } else {
                             // resolved directly -> store dataset name as a dependency
                             Pair<DataverseName, String> datasetReference = parseDatasetFunctionArguments(callExpr, 0);
-                            datasetDependencies
-                                    .add(new Triple<>(datasetReference.first, datasetReference.second, null));
+                            if (seenDatasets.add(datasetReference)) {
+                                datasetDependencies
+                                        .add(new Triple<>(datasetReference.first, datasetReference.second, null));
+                            }
                         }
                     } else if (BuiltinFunctions.getBuiltinFunctionInfo(signature.createFunctionIdentifier()) == null) {
-                        functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
-                                Integer.toString(signature.getArity())));
+                        if (seenFunctions.add(signature)) {
+                            functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
+                                    Integer.toString(signature.getArity())));
+                        }
                     }
                     break;
                 case WINDOW_EXPRESSION:
@@ -374,14 +344,35 @@
         return (value != null) && Boolean.parseBoolean(value.toLowerCase());
     }
 
-    public static Set<FunctionSignature> getFunctionSignatures(List<FunctionDecl> declaredFunctions) {
+    public static Map<FunctionSignature, FunctionDecl> getFunctionMap(List<FunctionDecl> declaredFunctions) {
         if (declaredFunctions == null || declaredFunctions.isEmpty()) {
-            return Collections.emptySet();
+            return Collections.emptyMap();
         }
-        Set<FunctionSignature> result = new HashSet<>();
+        Map<FunctionSignature, FunctionDecl> result = new HashMap<>();
         for (FunctionDecl fd : declaredFunctions) {
-            result.add(fd.getSignature());
+            result.put(fd.getSignature(), fd);
         }
         return result;
     }
+
+    public static FunctionDecl parseStoredFunction(Function function, IParserFactory parserFactory,
+            IWarningCollector warningCollector, SourceLocation sourceLoc) throws CompilationException {
+        if (!function.getLanguage().equals(parserFactory.getLanguage())) {
+            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE, sourceLoc,
+                    parserFactory.getLanguage(), function.getLanguage());
+        }
+        IParser parser = parserFactory.createParser(new StringReader(function.getFunctionBody()));
+        try {
+            FunctionDecl functionDecl =
+                    parser.parseFunctionBody(function.getSignature(), function.getParameterNames(), true);
+            functionDecl.setSourceLocation(sourceLoc);
+            if (warningCollector != null) {
+                parser.getWarnings(warningCollector);
+            }
+            return functionDecl;
+        } catch (CompilationException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, sourceLoc,
+                    function.getSignature(), e.getMessage());
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index f1bcb93..cba6bb5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -26,12 +26,9 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.base.IQueryRewriter;
-import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -58,31 +55,26 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-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;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, List<FunctionDecl>> {
+public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, Void> {
 
     protected final LangRewritingContext context;
-    protected final CloneAndSubstituteVariablesVisitor cloneVisitor;
-    private final IRewriterFactory rewriterFactory;
-    private final List<FunctionDecl> declaredFunctions;
-    private final MetadataProvider metadataProvider;
 
-    public AbstractInlineUdfsVisitor(LangRewritingContext context, IRewriterFactory rewriterFactory,
-            List<FunctionDecl> declaredFunctions, MetadataProvider metadataProvider,
+    protected final Map<FunctionSignature, FunctionDecl> usedUDFs;
+
+    protected final CloneAndSubstituteVariablesVisitor cloneVisitor;
+
+    public AbstractInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs,
             CloneAndSubstituteVariablesVisitor cloneVisitor) {
         this.context = context;
+        this.usedUDFs = usedUDFs;
         this.cloneVisitor = cloneVisitor;
-        this.rewriterFactory = rewriterFactory;
-        this.declaredFunctions = declaredFunctions;
-        this.metadataProvider = metadataProvider;
     }
 
     /**
@@ -96,36 +88,33 @@
             throws CompilationException;
 
     @Override
-    public Boolean visit(Query q, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(q.getBody(), arg);
+    public Boolean visit(Query q, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(q.getBody());
         q.setBody(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(FunctionDecl fd, List<FunctionDecl> arg) throws CompilationException {
-        // Careful, we should only do this after analyzing the graph of function
-        // calls.
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fd.getFuncBody(), arg);
-        fd.setFuncBody(p.second);
-        return p.first;
+    public Boolean visit(FunctionDecl fd, Void arg) throws CompilationException {
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fd.getSourceLocation(),
+                fd.getSignature().toString());
     }
 
     @Override
-    public Boolean visit(ListConstructor lc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(lc.getExprList(), arg);
+    public Boolean visit(ListConstructor lc, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(lc.getExprList());
         lc.setExprList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(RecordConstructor rc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(RecordConstructor rc, Void arg) throws CompilationException {
         boolean changed = false;
         for (FieldBinding b : rc.getFbList()) {
-            Pair<Boolean, Expression> leftExprInlined = inlineUdfsInExpr(b.getLeftExpr(), arg);
+            Pair<Boolean, Expression> leftExprInlined = inlineUdfsInExpr(b.getLeftExpr());
             b.setLeftExpr(leftExprInlined.second);
             changed = changed || leftExprInlined.first;
-            Pair<Boolean, Expression> rightExprInlined = inlineUdfsInExpr(b.getRightExpr(), arg);
+            Pair<Boolean, Expression> rightExprInlined = inlineUdfsInExpr(b.getRightExpr());
             b.setRightExpr(rightExprInlined.second);
             changed = changed || rightExprInlined.first;
         }
@@ -133,12 +122,12 @@
     }
 
     @Override
-    public Boolean visit(CallExpr callExpr, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(callExpr.getExprList(), arg);
+    public Boolean visit(CallExpr callExpr, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(callExpr.getExprList());
         callExpr.setExprList(p.second);
         boolean changed = p.first;
         if (callExpr.hasAggregateFilterExpr()) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(callExpr.getAggregateFilterExpr(), arg);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(callExpr.getAggregateFilterExpr());
             callExpr.setAggregateFilterExpr(be.second);
             changed |= be.first;
         }
@@ -146,96 +135,96 @@
     }
 
     @Override
-    public Boolean visit(OperatorExpr ifbo, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(ifbo.getExprList(), arg);
+    public Boolean visit(OperatorExpr ifbo, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(ifbo.getExprList());
         ifbo.setExprList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(FieldAccessor fa, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr(), arg);
+    public Boolean visit(FieldAccessor fa, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
         fa.setExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(IndexAccessor fa, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr(), arg);
+    public Boolean visit(IndexAccessor fa, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
         fa.setExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(IfExpr ifexpr, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(ifexpr.getCondExpr(), arg);
+    public Boolean visit(IfExpr ifexpr, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(ifexpr.getCondExpr());
         ifexpr.setCondExpr(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(ifexpr.getThenExpr(), arg);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(ifexpr.getThenExpr());
         ifexpr.setThenExpr(p2.second);
-        Pair<Boolean, Expression> p3 = inlineUdfsInExpr(ifexpr.getElseExpr(), arg);
+        Pair<Boolean, Expression> p3 = inlineUdfsInExpr(ifexpr.getElseExpr());
         ifexpr.setElseExpr(p3.second);
         return p1.first || p2.first || p3.first;
     }
 
     @Override
-    public Boolean visit(QuantifiedExpression qe, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(QuantifiedExpression qe, Void arg) throws CompilationException {
         boolean changed = false;
         for (QuantifiedPair t : qe.getQuantifiedList()) {
-            Pair<Boolean, Expression> p = inlineUdfsInExpr(t.getExpr(), arg);
+            Pair<Boolean, Expression> p = inlineUdfsInExpr(t.getExpr());
             t.setExpr(p.second);
             if (p.first) {
                 changed = true;
             }
         }
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(qe.getSatisfiesExpr(), arg);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(qe.getSatisfiesExpr());
         qe.setSatisfiesExpr(p2.second);
         return changed || p2.first;
     }
 
     @Override
-    public Boolean visit(LetClause lc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(lc.getBindingExpr(), arg);
+    public Boolean visit(LetClause lc, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(lc.getBindingExpr());
         lc.setBindingExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(WhereClause wc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(wc.getWhereExpr(), arg);
+    public Boolean visit(WhereClause wc, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(wc.getWhereExpr());
         wc.setWhereExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(OrderbyClause oc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(oc.getOrderbyList(), arg);
+    public Boolean visit(OrderbyClause oc, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(oc.getOrderbyList());
         oc.setOrderbyList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(GroupbyClause gc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(GroupbyClause gc, Void arg) throws CompilationException {
         boolean changed = false;
         List<List<GbyVariableExpressionPair>> gbyList = gc.getGbyPairList();
         List<List<GbyVariableExpressionPair>> newGbyList = new ArrayList<>(gbyList.size());
         for (List<GbyVariableExpressionPair> gbyPairList : gbyList) {
-            Pair<Boolean, List<GbyVariableExpressionPair>> p1 = inlineUdfsInGbyPairList(gbyPairList, arg);
+            Pair<Boolean, List<GbyVariableExpressionPair>> p1 = inlineUdfsInGbyPairList(gbyPairList);
             newGbyList.add(p1.second);
             changed |= p1.first;
         }
         gc.setGbyPairList(newGbyList);
         if (gc.hasDecorList()) {
-            Pair<Boolean, List<GbyVariableExpressionPair>> p2 = inlineUdfsInGbyPairList(gc.getDecorPairList(), arg);
+            Pair<Boolean, List<GbyVariableExpressionPair>> p2 = inlineUdfsInGbyPairList(gc.getDecorPairList());
             gc.setDecorPairList(p2.second);
             changed |= p2.first;
         }
         if (gc.hasGroupFieldList()) {
-            Pair<Boolean, List<Pair<Expression, Identifier>>> p3 = inlineUdfsInFieldList(gc.getGroupFieldList(), arg);
+            Pair<Boolean, List<Pair<Expression, Identifier>>> p3 = inlineUdfsInFieldList(gc.getGroupFieldList());
             gc.setGroupFieldList(p3.second);
             changed |= p3.first;
         }
         if (gc.hasWithMap()) {
-            Pair<Boolean, Map<Expression, VariableExpr>> p4 = inlineUdfsInVarMap(gc.getWithVarMap(), arg);
+            Pair<Boolean, Map<Expression, VariableExpr>> p4 = inlineUdfsInVarMap(gc.getWithVarMap());
             gc.setWithVarMap(p4.second);
             changed |= p4.first;
         }
@@ -243,15 +232,15 @@
     }
 
     @Override
-    public Boolean visit(LimitClause lc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(LimitClause lc, Void arg) throws CompilationException {
         boolean changed = false;
         if (lc.hasLimitExpr()) {
-            Pair<Boolean, Expression> p1 = inlineUdfsInExpr(lc.getLimitExpr(), arg);
+            Pair<Boolean, Expression> p1 = inlineUdfsInExpr(lc.getLimitExpr());
             lc.setLimitExpr(p1.second);
             changed = p1.first;
         }
         if (lc.hasOffset()) {
-            Pair<Boolean, Expression> p2 = inlineUdfsInExpr(lc.getOffset(), arg);
+            Pair<Boolean, Expression> p2 = inlineUdfsInExpr(lc.getOffset());
             lc.setOffset(p2.second);
             changed |= p2.first;
         }
@@ -259,131 +248,132 @@
     }
 
     @Override
-    public Boolean visit(UnaryExpr u, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(UnaryExpr u, Void arg) throws CompilationException {
         return u.getExpr().accept(this, arg);
     }
 
     @Override
-    public Boolean visit(VariableExpr v, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(VariableExpr v, Void arg) throws CompilationException {
         return false;
     }
 
     @Override
-    public Boolean visit(LiteralExpr l, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(LiteralExpr l, Void arg) throws CompilationException {
         return false;
     }
 
     @Override
-    public Boolean visit(InsertStatement insert, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(InsertStatement insert, Void arg) throws CompilationException {
         boolean changed = false;
         Expression returnExpression = insert.getReturnExpression();
         if (returnExpression != null) {
-            Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression, arg);
+            Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression);
             insert.setReturnExpression(rewrittenReturnExpr.second);
             changed |= rewrittenReturnExpr.first;
         }
-        Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody(), arg);
+        Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody());
         insert.setBody(rewrittenBodyExpression.second);
         return changed || rewrittenBodyExpression.first;
     }
 
-    protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr, List<FunctionDecl> arg)
-            throws CompilationException {
+    protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr) throws CompilationException {
         if (expr.getKind() != Kind.CALL_EXPRESSION) {
-            boolean r = expr.accept(this, arg);
+            boolean r = expr.accept(this, null);
             return new Pair<>(r, expr);
         }
         CallExpr f = (CallExpr) expr;
-        boolean r = expr.accept(this, arg);
-        FunctionDecl implem = findFuncDeclaration(f.getFunctionSignature(), arg);
-        if (implem == null) {
+        boolean r = expr.accept(this, null);
+        FunctionSignature fs = f.getFunctionSignature();
+        if (FunctionUtil.isBuiltinFunctionSignature(fs)) {
             return new Pair<>(r, expr);
-        } else {
-            if (f.hasAggregateFilterExpr()) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_FILTER_CLAUSE,
-                        f.getSourceLocation());
-            }
-            // Rewrite the function body itself (without setting unbounded variables to dataset access).
-            // TODO(buyingyi): throw an exception for recursive function definition or limit the stack depth.
-            implem.setFuncBody(rewriteFunctionBody(implem));
-            // it's one of the functions we want to inline
-            List<Expression> argList = f.getExprList();
-            int argCount = argList.size();
-            List<LetClause> clauses = new ArrayList<>(argCount + 1);
-            List<Expression> argVars = new ArrayList<>(argCount);
-            for (Expression e : f.getExprList()) {
-                // Obs: we could do smth about passing also literals, or let
-                // variable inlining to take care of this.
-                VarIdentifier argVar;
-                if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
-                    argVar = ((VariableExpr) e).getVar();
-                } else {
-                    SourceLocation sourceLoc = e.getSourceLocation();
-                    argVar = context.newVariable();
-                    Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
-                            e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
-                    VariableExpr newVRef1 = new VariableExpr(argVar);
-                    newVRef1.setSourceLocation(sourceLoc);
-                    LetClause c = new LetClause(newVRef1, (Expression) p1.first);
-                    c.setSourceLocation(sourceLoc);
-                    clauses.add(c);
-                }
-
-                VariableExpr argVarExpr = new VariableExpr(argVar);
-                argVarExpr.setSourceLocation(e.getSourceLocation());
-                argVars.add(argVarExpr);
-            }
-
-            VariableSubstitutionEnvironment subst = new VariableSubstitutionEnvironment();
-            List<VarIdentifier> paramList = implem.getParamList();
-            if (implem.getSignature().getArity() == FunctionIdentifier.VARARGS) {
-                if (paramList.size() != 1) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
-                            paramList.size());
-                }
-                VarIdentifier paramVarargs = paramList.get(0);
-                CallExpr argsListExpr =
-                        new CallExpr(new FunctionSignature(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR), argVars);
-                argsListExpr.setSourceLocation(expr.getSourceLocation());
-
-                VarIdentifier argsVar = context.newVariable();
-                VariableExpr argsVarRef1 = new VariableExpr(argsVar);
-                argsVarRef1.setSourceLocation(expr.getSourceLocation());
-                LetClause c = new LetClause(argsVarRef1, argsListExpr);
-                c.setSourceLocation(expr.getSourceLocation());
-                clauses.add(c);
-
-                VariableExpr argsVarRef2 = new VariableExpr(argsVar);
-                argsVarRef2.setSourceLocation(expr.getSourceLocation());
-                subst.addSubstituion(new VariableExpr(paramVarargs), argsVarRef2);
-            } else {
-                if (paramList.size() != argCount) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
-                            paramList.size());
-                }
-                for (int i = 0; i < argCount; i++) {
-                    subst.addSubstituion(new VariableExpr(paramList.get(i)), argVars.get(i));
-                }
-            }
-
-            Pair<ILangExpression, VariableSubstitutionEnvironment> p2 =
-                    implem.getFuncBody().accept(cloneVisitor, subst);
-            Expression resExpr;
-            if (clauses.isEmpty()) {
-                resExpr = (Expression) p2.first;
-            } else {
-                resExpr = generateQueryExpression(clauses, (Expression) p2.first);
-            }
-            return new Pair<>(true, resExpr);
         }
+        if (f.hasAggregateFilterExpr()) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_FILTER_CLAUSE, f.getSourceLocation());
+        }
+        FunctionDecl implem = usedUDFs.get(fs);
+        if (implem == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, f.getSourceLocation(), fs.toString());
+        }
+        // it's one of the functions we want to inline
+        List<Expression> argList = f.getExprList();
+        int argCount = argList.size();
+        List<LetClause> clauses = new ArrayList<>(argCount + 1);
+        List<Expression> argVars = new ArrayList<>(argCount);
+        for (Expression e : f.getExprList()) {
+            // Obs: we could do smth about passing also literals, or let
+            // variable inlining to take care of this.
+            VarIdentifier argVar;
+            if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
+                argVar = ((VariableExpr) e).getVar();
+            } else {
+                SourceLocation sourceLoc = e.getSourceLocation();
+                argVar = context.newVariable();
+                Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
+                        e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
+                VariableExpr newVRef1 = new VariableExpr(argVar);
+                newVRef1.setSourceLocation(sourceLoc);
+                LetClause c = new LetClause(newVRef1, (Expression) p1.first);
+                c.setSourceLocation(sourceLoc);
+                clauses.add(c);
+            }
+
+            VariableExpr argVarExpr = new VariableExpr(argVar);
+            argVarExpr.setSourceLocation(e.getSourceLocation());
+            argVars.add(argVarExpr);
+        }
+
+        VariableSubstitutionEnvironment subst = new VariableSubstitutionEnvironment();
+        List<VarIdentifier> paramList = implem.getParamList();
+        if (implem.getSignature().getArity() == FunctionIdentifier.VARARGS) {
+            if (paramList.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
+                        paramList.size());
+            }
+            VarIdentifier paramVarargs = paramList.get(0);
+            CallExpr argsListExpr =
+                    new CallExpr(new FunctionSignature(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR), argVars);
+            argsListExpr.setSourceLocation(expr.getSourceLocation());
+
+            VarIdentifier argsVar = context.newVariable();
+            VariableExpr argsVarRef1 = new VariableExpr(argsVar);
+            argsVarRef1.setSourceLocation(expr.getSourceLocation());
+            LetClause c = new LetClause(argsVarRef1, argsListExpr);
+            c.setSourceLocation(expr.getSourceLocation());
+            clauses.add(c);
+
+            VariableExpr argsVarRef2 = new VariableExpr(argsVar);
+            argsVarRef2.setSourceLocation(expr.getSourceLocation());
+            subst.addSubstituion(new VariableExpr(paramVarargs), argsVarRef2);
+        } else {
+            if (paramList.size() != argCount) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
+                        paramList.size());
+            }
+            for (int i = 0; i < argCount; i++) {
+                subst.addSubstituion(new VariableExpr(paramList.get(i)), argVars.get(i));
+            }
+        }
+
+        Expression funcBodyNorm = implem.getNormalizedFuncBody();
+        if (funcBodyNorm == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(), fs.toString());
+        }
+        Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = funcBodyNorm.accept(cloneVisitor, subst);
+        Expression resExpr;
+        if (clauses.isEmpty()) {
+            resExpr = (Expression) p2.first;
+        } else {
+            resExpr = generateQueryExpression(clauses, (Expression) p2.first);
+        }
+        return new Pair<>(true, resExpr);
     }
 
-    protected Pair<Boolean, List<Expression>> inlineUdfsInExprList(List<Expression> exprList, List<FunctionDecl> fds)
+    protected Pair<Boolean, List<Expression>> inlineUdfsInExprList(List<Expression> exprList)
             throws CompilationException {
         List<Expression> newList = new ArrayList<>(exprList.size());
         boolean changed = false;
         for (Expression e : exprList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(e, fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(e);
             newList.add(be.second);
             changed |= be.first;
         }
@@ -391,11 +381,11 @@
     }
 
     private Pair<Boolean, List<GbyVariableExpressionPair>> inlineUdfsInGbyPairList(
-            List<GbyVariableExpressionPair> gbyPairList, List<FunctionDecl> fds) throws CompilationException {
+            List<GbyVariableExpressionPair> gbyPairList) throws CompilationException {
         List<GbyVariableExpressionPair> newList = new ArrayList<>(gbyPairList.size());
         boolean changed = false;
         for (GbyVariableExpressionPair p : gbyPairList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.getExpr(), fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.getExpr());
             newList.add(new GbyVariableExpressionPair(p.getVar(), be.second));
             changed |= be.first;
         }
@@ -403,69 +393,26 @@
     }
 
     protected Pair<Boolean, List<Pair<Expression, Identifier>>> inlineUdfsInFieldList(
-            List<Pair<Expression, Identifier>> fieldList, List<FunctionDecl> fds) throws CompilationException {
+            List<Pair<Expression, Identifier>> fieldList) throws CompilationException {
         List<Pair<Expression, Identifier>> newList = new ArrayList<>(fieldList.size());
         boolean changed = false;
         for (Pair<Expression, Identifier> p : fieldList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.first, fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.first);
             newList.add(new Pair<>(be.second, p.second));
             changed |= be.first;
         }
         return new Pair<>(changed, newList);
     }
 
-    private Pair<Boolean, Map<Expression, VariableExpr>> inlineUdfsInVarMap(Map<Expression, VariableExpr> varMap,
-            List<FunctionDecl> fds) throws CompilationException {
+    private Pair<Boolean, Map<Expression, VariableExpr>> inlineUdfsInVarMap(Map<Expression, VariableExpr> varMap)
+            throws CompilationException {
         Map<Expression, VariableExpr> newMap = new HashMap<>();
         boolean changed = false;
         for (Map.Entry<Expression, VariableExpr> me : varMap.entrySet()) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(me.getKey(), fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(me.getKey());
             newMap.put(be.second, me.getValue());
             changed |= be.first;
         }
         return new Pair<>(changed, newMap);
     }
-
-    private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
-        SourceLocation sourceLoc = fnDecl.getSourceLocation();
-
-        DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
-        Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
-        Dataverse fnDataverse;
-        if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
-            fnDataverse = defaultDataverse;
-        } else {
-            try {
-                fnDataverse = metadataProvider.findDataverse(fnDataverseName);
-            } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnDataverseName);
-            }
-        }
-
-        metadataProvider.setDefaultDataverse(fnDataverse);
-        try {
-            Query wrappedQuery = new Query(false);
-            wrappedQuery.setSourceLocation(sourceLoc);
-            wrappedQuery.setBody(fnDecl.getFuncBody());
-            wrappedQuery.setTopLevel(false);
-            IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
-            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context, true,
-                    fnDecl.getParamList());
-            return wrappedQuery.getBody();
-        } catch (CompilationException e) {
-            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, fnDecl.getSignature(),
-                    e.getMessage());
-        } finally {
-            metadataProvider.setDefaultDataverse(defaultDataverse);
-        }
-    }
-
-    private static FunctionDecl findFuncDeclaration(FunctionSignature fid, List<FunctionDecl> sequence) {
-        for (FunctionDecl f : sequence) {
-            if (f.getSignature().equals(fid)) {
-                return f;
-            }
-        }
-        return null;
-    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index 1fcf822..4b30c97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -178,7 +178,7 @@
         }
 
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = fd.getFuncBody().accept(this, env);
-        FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first);
+        FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first, fd.isStored());
         newF.setSourceLocation(fd.getSourceLocation());
         return new Pair<>(newF, env);
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index 3ad0f1b..25f4103 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -19,10 +19,9 @@
 
 package org.apache.asterix.lang.common.visitor;
 
-import java.util.LinkedHashSet;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.Expression;
@@ -56,9 +55,13 @@
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class GatherFunctionCallsVisitor extends AbstractQueryExpressionVisitor<Void, Void> {
+public abstract class GatherFunctionCallsVisitor extends AbstractQueryExpressionVisitor<Void, Void> {
 
-    protected final Set<AbstractCallExpression> calls = new LinkedHashSet<>();
+    protected final Collection<? super AbstractCallExpression> calls;
+
+    protected GatherFunctionCallsVisitor(Collection<? super AbstractCallExpression> calls) {
+        this.calls = calls;
+    }
 
     @Override
     public Void visit(CallExpr callExpr, Void arg) throws CompilationException {
@@ -240,10 +243,6 @@
         return null;
     }
 
-    public Set<AbstractCallExpression> getCalls() {
-        return calls;
-    }
-
     @Override
     public Void visit(FunctionDecl fd, Void arg) throws CompilationException {
         return null;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 356a683..8dd206f 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -19,15 +19,13 @@
 package org.apache.asterix.lang.sqlpp.rewrites;
 
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
 
@@ -36,11 +34,16 @@
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars) throws CompilationException {
+    public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
+        if (inlineUdfs) {
+            // When rewriting function body we do not inline UDFs into it.
+            // The main query rewriter will inline everything later, when it processes the query
+            throw new CompilationException(ErrorCode.ILLEGAL_STATE, topStatement.getSourceLocation(), "inlineUdfs");
+        }
+
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
+        setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
 
         // Resolves function calls
         resolveFunctionCalls();
@@ -93,8 +96,5 @@
 
         // Rewrites RIGHT OUTER JOINs into LEFT OUTER JOINs if possible
         rewriteRightJoins();
-
-        // Inlines functions recursively.
-        inlineDeclaredUdfs(inlineUdfs);
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index af1bb41..738fc69 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -18,48 +18,31 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites;
 
-import java.util.ArrayList;
+import java.util.ArrayDeque;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashSet;
-import java.util.List;
+import java.util.LinkedHashMap;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IReturningStatement;
-import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
-import org.apache.asterix.lang.common.expression.ListSliceExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
-import org.apache.asterix.lang.common.parser.FunctionParser;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
-import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.FromTerm;
-import org.apache.asterix.lang.sqlpp.clause.HavingClause;
-import org.apache.asterix.lang.sqlpp.clause.JoinClause;
-import org.apache.asterix.lang.sqlpp.clause.NestClause;
-import org.apache.asterix.lang.sqlpp.clause.Projection;
-import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
-import org.apache.asterix.lang.sqlpp.clause.SelectClause;
-import org.apache.asterix.lang.sqlpp.clause.SelectElement;
-import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
-import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
-import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
-import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
-import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
-import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
@@ -68,6 +51,7 @@
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseAggregateExtractionVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseExpressionVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppFunctionCallResolverVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGatherFunctionCallsVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByAggregationSugarVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupingSetsVisitor;
@@ -79,12 +63,12 @@
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppWindowRewriteVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SubstituteGroupbyExpressionWithVariableVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.VariableCheckAndRewriteVisitor;
-import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
-import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.Function;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.util.LogRedactionUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -96,41 +80,38 @@
     public static final String INLINE_WITH_OPTION = "inline_with";
     private static final boolean INLINE_WITH_OPTION_DEFAULT = true;
     private final IParserFactory parserFactory;
-    private final FunctionParser functionParser;
-    private IReturningStatement topExpr;
-    private List<FunctionDecl> declaredFunctions;
+    private SqlppFunctionBodyRewriter functionBodyRewriter;
+    private IReturningStatement topStatement;
     private LangRewritingContext context;
     private MetadataProvider metadataProvider;
     private Collection<VarIdentifier> externalVars;
+    private boolean allowNonStoredUdfCalls;
+    private boolean inlineUdfs;
     private boolean isLogEnabled;
 
     public SqlppQueryRewriter(IParserFactory parserFactory) {
         this.parserFactory = parserFactory;
-        functionParser = new FunctionParser(parserFactory);
     }
 
-    protected void setup(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context, Collection<VarIdentifier> externalVars)
+    protected void setup(LangRewritingContext context, IReturningStatement topStatement,
+            Collection<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls, boolean inlineUdfs)
             throws CompilationException {
-        this.topExpr = topExpr;
         this.context = context;
-        this.declaredFunctions = declaredFunctions;
-        this.metadataProvider = metadataProvider;
+        this.metadataProvider = context.getMetadataProvider();
+        this.topStatement = topStatement;
         this.externalVars = externalVars != null ? externalVars : Collections.emptyList();
+        this.allowNonStoredUdfCalls = allowNonStoredUdfCalls;
+        this.inlineUdfs = inlineUdfs;
         this.isLogEnabled = LOGGER.isTraceEnabled();
         logExpression("Starting AST rewrites on", "");
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars) throws CompilationException {
-        if (topStatement == null) {
-            return;
-        }
+    public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
 
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
+        setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
 
         // Resolves function calls
         resolveFunctionCalls();
@@ -182,7 +163,7 @@
         rewriteRightJoins();
 
         // Inlines functions.
-        inlineDeclaredUdfs(inlineUdfs);
+        loadAndInlineDeclaredUdfs();
 
         // Rewrites SQL++ core aggregate function names into internal names
         rewriteSpecialFunctionNames();
@@ -214,7 +195,7 @@
 
     protected void resolveFunctionCalls() throws CompilationException {
         SqlppFunctionCallResolverVisitor visitor =
-                new SqlppFunctionCallResolverVisitor(metadataProvider, declaredFunctions);
+                new SqlppFunctionCallResolverVisitor(context, allowNonStoredUdfCalls);
         rewriteTopExpr(visitor, null);
     }
 
@@ -308,53 +289,40 @@
         rewriteTopExpr(visitor, null);
     }
 
-    protected void inlineDeclaredUdfs(boolean inlineUdfs) throws CompilationException {
-        List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
-        for (FunctionDecl fdecl : declaredFunctions) {
-            funIds.add(fdecl.getSignature());
-        }
-
-        List<FunctionDecl> usedStoredFunctionDecls = new ArrayList<>();
-        for (Expression topLevelExpr : topExpr.getDirectlyEnclosedExpressions()) {
-            usedStoredFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr,
-                    funIds, null, this::getFunctionCalls, functionParser, context.getWarningCollector()));
-        }
-        declaredFunctions.addAll(usedStoredFunctionDecls);
-        if (inlineUdfs && !declaredFunctions.isEmpty()) {
-            SqlppFunctionBodyRewriterFactory functionBodyRewriterFactory =
-                    new SqlppFunctionBodyRewriterFactory(parserFactory);
-            SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, functionBodyRewriterFactory,
-                    declaredFunctions, metadataProvider);
-            while (rewriteTopExpr(visitor, declaredFunctions)) {
+    protected void loadAndInlineDeclaredUdfs() throws CompilationException {
+        Map<FunctionSignature, FunctionDecl> udfs = fetchUserDefinedFunctions(topStatement);
+        FunctionUtil.checkFunctionRecursion(udfs, SqlppGatherFunctionCallsVisitor::new,
+                topStatement.getSourceLocation());
+        if (!udfs.isEmpty() && inlineUdfs) {
+            SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, udfs);
+            while (rewriteTopExpr(visitor, null)) {
                 // loop until no more changes
             }
         }
-        declaredFunctions.removeAll(usedStoredFunctionDecls);
     }
 
     private <R, T> R rewriteTopExpr(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
-        R result = topExpr.accept(visitor, arg);
+        R result = topStatement.accept(visitor, arg);
         logExpression(">>>> AST After", visitor.getClass().getSimpleName());
         return result;
     }
 
     private void logExpression(String p0, String p1) throws CompilationException {
         if (isLogEnabled) {
-            LOGGER.trace("{} {}\n{}", p0, p1, LogRedactionUtil.userData(SqlppAstPrintUtil.toString(topExpr)));
+            LOGGER.trace("{} {}\n{}", p0, p1, LogRedactionUtil.userData(SqlppAstPrintUtil.toString(topStatement)));
         }
     }
 
     @Override
-    public Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException {
-        GatherFunctionCalls gfc = new GatherFunctionCalls();
+    public void getFunctionCalls(Expression expression, Collection<? super AbstractCallExpression> outCalls)
+            throws CompilationException {
+        SqlppGatherFunctionCallsVisitor gfc = new SqlppGatherFunctionCallsVisitor(outCalls);
         expression.accept(gfc, null);
-        return gfc.getCalls();
     }
 
     @Override
     public Set<VariableExpr> getExternalVariables(Expression expr) throws CompilationException {
         Set<VariableExpr> freeVars = SqlppVariableUtil.getFreeVariables(expr);
-
         Set<VariableExpr> extVars = new HashSet<>();
         for (VariableExpr ve : freeVars) {
             if (SqlppVariableUtil.isExternalVariableReference(ve)) {
@@ -364,189 +332,104 @@
         return extVars;
     }
 
-    private static class GatherFunctionCalls extends GatherFunctionCallsVisitor implements ISqlppVisitor<Void, Void> {
+    private Map<FunctionSignature, FunctionDecl> fetchUserDefinedFunctions(IReturningStatement topExpr)
+            throws CompilationException {
+        Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
 
-        public GatherFunctionCalls() {
+        Deque<AbstractCallExpression> workQueue = new ArrayDeque<>();
+        SqlppGatherFunctionCallsVisitor gfc = new SqlppGatherFunctionCallsVisitor(workQueue);
+        for (Expression expr : topExpr.getDirectlyEnclosedExpressions()) {
+            expr.accept(gfc, null);
+        }
+        AbstractCallExpression fnCall;
+        while ((fnCall = workQueue.poll()) != null) {
+            switch (fnCall.getKind()) {
+                case CALL_EXPRESSION:
+                    FunctionSignature fs = fnCall.getFunctionSignature();
+                    DataverseName fsDataverse = fs.getDataverseName();
+                    if (fsDataverse == null) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fnCall.getSourceLocation(),
+                                fs);
+                    }
+                    if (FunctionUtil.isBuiltinFunctionSignature(fs) || udfs.containsKey(fs)) {
+                        continue;
+                    }
+                    FunctionDecl fd = context.getDeclaredFunctions().get(fs);
+                    if (fd == null) {
+                        Function function;
+                        try {
+                            function = metadataProvider.lookupUserDefinedFunction(fs);
+                        } catch (AlgebricksException e) {
+                            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
+                                    fs.toString());
+                        }
+                        if (function == null) {
+                            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
+                                    fs.toString());
+                        }
+                        fd = FunctionUtil.parseStoredFunction(function, parserFactory, context.getWarningCollector(),
+                                fnCall.getSourceLocation());
+                    }
+                    prepareFunction(fd);
+                    udfs.put(fs, fd);
+                    fd.getNormalizedFuncBody().accept(gfc, null);
+                    break;
+                case WINDOW_EXPRESSION:
+                    // there cannot be used-defined window functions
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fnCall.getSourceLocation(),
+                            fnCall.getFunctionSignature().toString(false));
+            }
+        }
+        return udfs;
+    }
+
+    private void prepareFunction(FunctionDecl fd) throws CompilationException {
+        Expression fnNormBody = fd.getNormalizedFuncBody();
+        if (fnNormBody == null) {
+            fnNormBody = rewriteFunctionBody(fd);
+            fd.setNormalizedFuncBody(fnNormBody);
+        }
+    }
+
+    private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
+        DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
+        Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
+        Dataverse fnDataverse;
+        if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
+            fnDataverse = defaultDataverse;
+        } else {
+            try {
+                fnDataverse = metadataProvider.findDataverse(fnDataverseName);
+            } catch (AlgebricksException e) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, fnDecl.getSourceLocation(),
+                        fnDataverseName);
+            }
         }
 
-        @Override
-        public Void visit(FromClause fromClause, Void arg) throws CompilationException {
-            for (FromTerm fromTerm : fromClause.getFromTerms()) {
-                fromTerm.accept(this, arg);
-            }
-            return null;
+        metadataProvider.setDefaultDataverse(fnDataverse);
+        try {
+            Query wrappedQuery = new Query(false);
+            wrappedQuery.setSourceLocation(fnDecl.getSourceLocation());
+            wrappedQuery.setBody(fnDecl.getFuncBody());
+            wrappedQuery.setTopLevel(false);
+            boolean allowNonStoredUdfCalls = !fnDecl.isStored();
+            getFunctionBodyRewriter().rewrite(context, wrappedQuery, allowNonStoredUdfCalls, false,
+                    fnDecl.getParamList());
+            return wrappedQuery.getBody();
+        } catch (CompilationException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, fnDecl.getSignature(),
+                    e.getMessage());
+        } finally {
+            metadataProvider.setDefaultDataverse(defaultDataverse);
         }
+    }
 
-        @Override
-        public Void visit(FromTerm fromTerm, Void arg) throws CompilationException {
-            fromTerm.getLeftExpression().accept(this, arg);
-            for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-                correlateClause.accept(this, arg);
-            }
-            return null;
+    protected SqlppFunctionBodyRewriter getFunctionBodyRewriter() {
+        if (functionBodyRewriter == null) {
+            functionBodyRewriter = new SqlppFunctionBodyRewriter(parserFactory);
         }
-
-        @Override
-        public Void visit(JoinClause joinClause, Void arg) throws CompilationException {
-            joinClause.getRightExpression().accept(this, arg);
-            joinClause.getConditionExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(NestClause nestClause, Void arg) throws CompilationException {
-            nestClause.getRightExpression().accept(this, arg);
-            nestClause.getConditionExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(Projection projection, Void arg) throws CompilationException {
-            if (!projection.star()) {
-                projection.getExpression().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectBlock selectBlock, Void arg) throws CompilationException {
-            if (selectBlock.hasFromClause()) {
-                selectBlock.getFromClause().accept(this, arg);
-            }
-            if (selectBlock.hasLetWhereClauses()) {
-                for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
-                    letWhereClause.accept(this, arg);
-                }
-            }
-            if (selectBlock.hasGroupbyClause()) {
-                selectBlock.getGroupbyClause().accept(this, arg);
-            }
-            if (selectBlock.hasLetHavingClausesAfterGroupby()) {
-                for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
-                    letHavingClause.accept(this, arg);
-                }
-            }
-            selectBlock.getSelectClause().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectClause selectClause, Void arg) throws CompilationException {
-            if (selectClause.selectElement()) {
-                selectClause.getSelectElement().accept(this, arg);
-            } else {
-                selectClause.getSelectRegular().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectElement selectElement, Void arg) throws CompilationException {
-            selectElement.getExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectRegular selectRegular, Void arg) throws CompilationException {
-            for (Projection projection : selectRegular.getProjections()) {
-                projection.accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
-            selectSetOperation.getLeftInput().accept(this, arg);
-            for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
-                setOperationRight.getSetOperationRightInput().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectExpression selectStatement, Void arg) throws CompilationException {
-            if (selectStatement.hasLetClauses()) {
-                for (LetClause letClause : selectStatement.getLetList()) {
-                    letClause.accept(this, arg);
-                }
-            }
-            selectStatement.getSelectSetOperation().accept(this, arg);
-            if (selectStatement.hasOrderby()) {
-                selectStatement.getOrderbyClause().accept(this, arg);
-            }
-            if (selectStatement.hasLimit()) {
-                selectStatement.getLimitClause().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(UnnestClause unnestClause, Void arg) throws CompilationException {
-            unnestClause.getRightExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(HavingClause havingClause, Void arg) throws CompilationException {
-            havingClause.getFilterExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(CaseExpression caseExpression, Void arg) throws CompilationException {
-            caseExpression.getConditionExpr().accept(this, arg);
-            for (Expression expr : caseExpression.getWhenExprs()) {
-                expr.accept(this, arg);
-            }
-            for (Expression expr : caseExpression.getThenExprs()) {
-                expr.accept(this, arg);
-            }
-            caseExpression.getElseExpr().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(WindowExpression winExpr, Void arg) throws CompilationException {
-            calls.add(winExpr);
-            if (winExpr.hasPartitionList()) {
-                for (Expression expr : winExpr.getPartitionList()) {
-                    expr.accept(this, arg);
-                }
-            }
-            if (winExpr.hasOrderByList()) {
-                for (Expression expr : winExpr.getOrderbyList()) {
-                    expr.accept(this, arg);
-                }
-            }
-            if (winExpr.hasFrameStartExpr()) {
-                winExpr.getFrameStartExpr().accept(this, arg);
-            }
-            if (winExpr.hasFrameEndExpr()) {
-                winExpr.getFrameEndExpr().accept(this, arg);
-            }
-            if (winExpr.hasWindowFieldList()) {
-                for (Pair<Expression, Identifier> p : winExpr.getWindowFieldList()) {
-                    p.first.accept(this, arg);
-                }
-            }
-            if (winExpr.hasAggregateFilterExpr()) {
-                winExpr.getAggregateFilterExpr().accept(this, arg);
-            }
-            for (Expression expr : winExpr.getExprList()) {
-                expr.accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(ListSliceExpression expression, Void arg) throws CompilationException {
-            expression.getExpr().accept(this, arg);
-            expression.getStartIndexExpression().accept(this, arg);
-
-            if (expression.hasEndExpression()) {
-                expression.getEndIndexExpression().accept(this, arg);
-            }
-            return null;
-        }
+        return functionBodyRewriter;
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
index 7666032..38b66e2 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
-import java.util.List;
-import java.util.Set;
 import java.util.function.BiFunction;
 
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -28,35 +26,35 @@
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.expression.CallExpr;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 
 public final class SqlppFunctionCallResolverVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
-    private final MetadataProvider metadataProvider;
+    private final LangRewritingContext context;
 
-    private final Set<FunctionSignature> declaredFunctions;
+    private final boolean allowNonStoredUdfCalls;
 
     private final BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver;
 
     private final BiFunction<String, Integer, FunctionSignature> callExprResolver;
 
-    public SqlppFunctionCallResolverVisitor(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions) {
-        this.metadataProvider = metadataProvider;
-        this.declaredFunctions = FunctionUtil.getFunctionSignatures(declaredFunctions);
-        this.builtinFunctionResolver = FunctionUtil.createBuiltinFunctionResolver(metadataProvider);
+    public SqlppFunctionCallResolverVisitor(LangRewritingContext context, boolean allowNonStoredUdfCalls) {
+        this.context = context;
+        this.allowNonStoredUdfCalls = allowNonStoredUdfCalls;
+        this.builtinFunctionResolver = FunctionUtil.createBuiltinFunctionResolver(context.getMetadataProvider());
         this.callExprResolver = this::resolveCallExpr;
     }
 
     @Override
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         FunctionSignature fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(),
-                callExpr.getSourceLocation(), metadataProvider, declaredFunctions, callExprResolver);
+                callExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, true,
+                context.getDeclaredFunctions(), allowNonStoredUdfCalls);
         callExpr.setFunctionSignature(fs);
         return super.visit(callExpr, arg);
     }
@@ -64,7 +62,7 @@
     @Override
     public Expression visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
         FunctionSignature fs = FunctionUtil.resolveFunctionCall(winExpr.getFunctionSignature(),
-                winExpr.getSourceLocation(), metadataProvider, declaredFunctions, callExprResolver);
+                winExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, false, null, false);
         winExpr.setFunctionSignature(fs);
         return super.visit(winExpr, arg);
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
new file mode 100644
index 0000000..8559c96
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
@@ -0,0 +1,238 @@
+/*
+ * 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.lang.sqlpp.rewrites.visitor;
+
+import java.util.Collection;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.expression.AbstractCallExpression;
+import org.apache.asterix.lang.common.expression.ListSliceExpression;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
+import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.HavingClause;
+import org.apache.asterix.lang.sqlpp.clause.JoinClause;
+import org.apache.asterix.lang.sqlpp.clause.NestClause;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
+import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
+import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public final class SqlppGatherFunctionCallsVisitor extends GatherFunctionCallsVisitor
+        implements ISqlppVisitor<Void, Void> {
+
+    public SqlppGatherFunctionCallsVisitor(Collection<? super AbstractCallExpression> calls) {
+        super(calls);
+    }
+
+    @Override
+    public Void visit(FromClause fromClause, Void arg) throws CompilationException {
+        for (FromTerm fromTerm : fromClause.getFromTerms()) {
+            fromTerm.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(FromTerm fromTerm, Void arg) throws CompilationException {
+        fromTerm.getLeftExpression().accept(this, arg);
+        for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
+            correlateClause.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(JoinClause joinClause, Void arg) throws CompilationException {
+        joinClause.getRightExpression().accept(this, arg);
+        joinClause.getConditionExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(NestClause nestClause, Void arg) throws CompilationException {
+        nestClause.getRightExpression().accept(this, arg);
+        nestClause.getConditionExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(Projection projection, Void arg) throws CompilationException {
+        if (!projection.star()) {
+            projection.getExpression().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectBlock selectBlock, Void arg) throws CompilationException {
+        if (selectBlock.hasFromClause()) {
+            selectBlock.getFromClause().accept(this, arg);
+        }
+        if (selectBlock.hasLetWhereClauses()) {
+            for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
+                letWhereClause.accept(this, arg);
+            }
+        }
+        if (selectBlock.hasGroupbyClause()) {
+            selectBlock.getGroupbyClause().accept(this, arg);
+        }
+        if (selectBlock.hasLetHavingClausesAfterGroupby()) {
+            for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
+                letHavingClause.accept(this, arg);
+            }
+        }
+        selectBlock.getSelectClause().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectClause selectClause, Void arg) throws CompilationException {
+        if (selectClause.selectElement()) {
+            selectClause.getSelectElement().accept(this, arg);
+        } else {
+            selectClause.getSelectRegular().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectElement selectElement, Void arg) throws CompilationException {
+        selectElement.getExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectRegular selectRegular, Void arg) throws CompilationException {
+        for (Projection projection : selectRegular.getProjections()) {
+            projection.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
+        selectSetOperation.getLeftInput().accept(this, arg);
+        for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
+            setOperationRight.getSetOperationRightInput().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectExpression selectStatement, Void arg) throws CompilationException {
+        if (selectStatement.hasLetClauses()) {
+            for (LetClause letClause : selectStatement.getLetList()) {
+                letClause.accept(this, arg);
+            }
+        }
+        selectStatement.getSelectSetOperation().accept(this, arg);
+        if (selectStatement.hasOrderby()) {
+            selectStatement.getOrderbyClause().accept(this, arg);
+        }
+        if (selectStatement.hasLimit()) {
+            selectStatement.getLimitClause().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(UnnestClause unnestClause, Void arg) throws CompilationException {
+        unnestClause.getRightExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(HavingClause havingClause, Void arg) throws CompilationException {
+        havingClause.getFilterExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(CaseExpression caseExpression, Void arg) throws CompilationException {
+        caseExpression.getConditionExpr().accept(this, arg);
+        for (Expression expr : caseExpression.getWhenExprs()) {
+            expr.accept(this, arg);
+        }
+        for (Expression expr : caseExpression.getThenExprs()) {
+            expr.accept(this, arg);
+        }
+        caseExpression.getElseExpr().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(WindowExpression winExpr, Void arg) throws CompilationException {
+        calls.add(winExpr);
+        if (winExpr.hasPartitionList()) {
+            for (Expression expr : winExpr.getPartitionList()) {
+                expr.accept(this, arg);
+            }
+        }
+        if (winExpr.hasOrderByList()) {
+            for (Expression expr : winExpr.getOrderbyList()) {
+                expr.accept(this, arg);
+            }
+        }
+        if (winExpr.hasFrameStartExpr()) {
+            winExpr.getFrameStartExpr().accept(this, arg);
+        }
+        if (winExpr.hasFrameEndExpr()) {
+            winExpr.getFrameEndExpr().accept(this, arg);
+        }
+        if (winExpr.hasWindowFieldList()) {
+            for (Pair<Expression, Identifier> p : winExpr.getWindowFieldList()) {
+                p.first.accept(this, arg);
+            }
+        }
+        if (winExpr.hasAggregateFilterExpr()) {
+            winExpr.getAggregateFilterExpr().accept(this, arg);
+        }
+        for (Expression expr : winExpr.getExprList()) {
+            expr.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(ListSliceExpression expression, Void arg) throws CompilationException {
+        expression.getExpr().accept(this, arg);
+        expression.getStartIndexExpression().accept(this, arg);
+
+        if (expression.hasEndExpression()) {
+            expression.getEndIndexExpression().accept(this, arg);
+        }
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
index 4fb17f0..5200b94 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
@@ -23,9 +23,9 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
-import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.ListSliceExpression;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -52,26 +52,18 @@
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppCloneAndSubstituteVariablesVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor
-        implements ISqlppVisitor<Boolean, List<FunctionDecl>> {
+public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor implements ISqlppVisitor<Boolean, Void> {
 
     /**
      * @param context,
      *            manages ids of variables and guarantees uniqueness of variables.
-     * @param rewriterFactory,
-     *            a rewrite factory for rewriting user-defined functions.
-     * @param declaredFunctions,
-     *            a list of declared functions associated with the query.
-     * @param metadataProvider,
-     *            providing the definition of created (i.e., stored) user-defined functions.
+     * @param usedUDFs,
+     *            user defined functions used by this query.
      */
-    public SqlppInlineUdfsVisitor(LangRewritingContext context, IRewriterFactory rewriterFactory,
-            List<FunctionDecl> declaredFunctions, MetadataProvider metadataProvider) {
-        super(context, rewriterFactory, declaredFunctions, metadataProvider,
-                new SqlppCloneAndSubstituteVariablesVisitor(context));
+    public SqlppInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs) {
+        super(context, usedUDFs, new SqlppCloneAndSubstituteVariablesVisitor(context));
     }
 
     @Override
@@ -82,220 +74,218 @@
     }
 
     @Override
-    public Boolean visit(FromClause fromClause, List<FunctionDecl> func) throws CompilationException {
+    public Boolean visit(FromClause fromClause, Void arg) throws CompilationException {
         boolean changed = false;
         for (FromTerm fromTerm : fromClause.getFromTerms()) {
-            changed |= fromTerm.accept(this, func);
+            changed |= fromTerm.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(FromTerm fromTerm, List<FunctionDecl> func) throws CompilationException {
+    public Boolean visit(FromTerm fromTerm, Void arg) throws CompilationException {
         boolean changed = false;
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fromTerm.getLeftExpression(), func);
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fromTerm.getLeftExpression());
         fromTerm.setLeftExpression(p.second);
         changed |= p.first;
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-            changed |= correlateClause.accept(this, func);
+            changed |= correlateClause.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(JoinClause joinClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(joinClause.getRightExpression(), funcs);
+    public Boolean visit(JoinClause joinClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(joinClause.getRightExpression());
         joinClause.setRightExpression(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(joinClause.getConditionExpression(), funcs);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(joinClause.getConditionExpression());
         joinClause.setConditionExpression(p2.second);
         return p1.first || p2.first;
     }
 
     @Override
-    public Boolean visit(NestClause nestClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(nestClause.getRightExpression(), funcs);
+    public Boolean visit(NestClause nestClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(nestClause.getRightExpression());
         nestClause.setRightExpression(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(nestClause.getConditionExpression(), funcs);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(nestClause.getConditionExpression());
         nestClause.setConditionExpression(p2.second);
         return p1.first || p2.first;
     }
 
     @Override
-    public Boolean visit(Projection projection, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(Projection projection, Void arg) throws CompilationException {
         if (projection.star()) {
             return false;
         }
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(projection.getExpression(), funcs);
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(projection.getExpression());
         projection.setExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(SelectBlock selectBlock, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectBlock selectBlock, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectBlock.hasFromClause()) {
-            changed |= selectBlock.getFromClause().accept(this, funcs);
+            changed |= selectBlock.getFromClause().accept(this, arg);
         }
         if (selectBlock.hasLetWhereClauses()) {
             for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
-                changed |= letWhereClause.accept(this, funcs);
+                changed |= letWhereClause.accept(this, arg);
             }
         }
         if (selectBlock.hasGroupbyClause()) {
-            changed |= selectBlock.getGroupbyClause().accept(this, funcs);
+            changed |= selectBlock.getGroupbyClause().accept(this, arg);
         }
         if (selectBlock.hasLetHavingClausesAfterGroupby()) {
             for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
-                changed |= letHavingClause.accept(this, funcs);
+                changed |= letHavingClause.accept(this, arg);
             }
         }
-        changed |= selectBlock.getSelectClause().accept(this, funcs);
+        changed |= selectBlock.getSelectClause().accept(this, arg);
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectClause selectClause, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectClause selectClause, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectClause.selectElement()) {
-            changed |= selectClause.getSelectElement().accept(this, funcs);
+            changed |= selectClause.getSelectElement().accept(this, arg);
         } else {
-            changed |= selectClause.getSelectRegular().accept(this, funcs);
+            changed |= selectClause.getSelectRegular().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectElement selectElement, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(selectElement.getExpression(), funcs);
+    public Boolean visit(SelectElement selectElement, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(selectElement.getExpression());
         selectElement.setExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(SelectRegular selectRegular, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectRegular selectRegular, Void arg) throws CompilationException {
         boolean changed = false;
         for (Projection projection : selectRegular.getProjections()) {
-            changed |= projection.accept(this, funcs);
+            changed |= projection.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectSetOperation selectSetOperation, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
         boolean changed = false;
-        changed |= selectSetOperation.getLeftInput().accept(this, funcs);
+        changed |= selectSetOperation.getLeftInput().accept(this, arg);
         for (SetOperationRight right : selectSetOperation.getRightInputs()) {
-            changed |= right.getSetOperationRightInput().accept(this, funcs);
+            changed |= right.getSetOperationRightInput().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectExpression selectExpression, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectExpression selectExpression, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectExpression.hasLetClauses()) {
             for (LetClause letClause : selectExpression.getLetList()) {
-                changed |= letClause.accept(this, funcs);
+                changed |= letClause.accept(this, arg);
             }
         }
-        changed |= selectExpression.getSelectSetOperation().accept(this, funcs);
+        changed |= selectExpression.getSelectSetOperation().accept(this, arg);
         if (selectExpression.hasOrderby()) {
-            changed |= selectExpression.getOrderbyClause().accept(this, funcs);
+            changed |= selectExpression.getOrderbyClause().accept(this, arg);
         }
         if (selectExpression.hasLimit()) {
-            changed |= selectExpression.getLimitClause().accept(this, funcs);
+            changed |= selectExpression.getLimitClause().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(UnnestClause unnestClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(unnestClause.getRightExpression(), funcs);
+    public Boolean visit(UnnestClause unnestClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(unnestClause.getRightExpression());
         unnestClause.setRightExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(HavingClause havingClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(havingClause.getFilterExpression(), funcs);
+    public Boolean visit(HavingClause havingClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(havingClause.getFilterExpression());
         havingClause.setFilterExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(CaseExpression caseExpr, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> result = inlineUdfsInExpr(caseExpr.getConditionExpr(), funcs);
+    public Boolean visit(CaseExpression caseExpr, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> result = inlineUdfsInExpr(caseExpr.getConditionExpr());
         caseExpr.setConditionExpr(result.second);
         boolean inlined = result.first;
 
-        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(caseExpr.getWhenExprs(), funcs);
+        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(caseExpr.getWhenExprs());
         inlined = inlined || inlinedList.first;
         caseExpr.setWhenExprs(inlinedList.second);
 
-        inlinedList = inlineUdfsInExprList(caseExpr.getThenExprs(), funcs);
+        inlinedList = inlineUdfsInExprList(caseExpr.getThenExprs());
         inlined = inlined || inlinedList.first;
         caseExpr.setThenExprs(inlinedList.second);
 
-        result = inlineUdfsInExpr(caseExpr.getElseExpr(), funcs);
+        result = inlineUdfsInExpr(caseExpr.getElseExpr());
         caseExpr.setElseExpr(result.second);
         return inlined || result.first;
     }
 
     @Override
-    public Boolean visit(WindowExpression winExpr, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(WindowExpression winExpr, Void arg) throws CompilationException {
         boolean inlined = false;
         if (winExpr.hasPartitionList()) {
-            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getPartitionList(), funcs);
+            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getPartitionList());
             winExpr.setPartitionList(inlinedList.second);
             inlined = inlinedList.first;
         }
         if (winExpr.hasOrderByList()) {
-            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getOrderbyList(), funcs);
+            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getOrderbyList());
             winExpr.setOrderbyList(inlinedList.second);
             inlined |= inlinedList.first;
         }
         if (winExpr.hasFrameStartExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameStartExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameStartExpr());
             winExpr.setFrameStartExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
         if (winExpr.hasFrameEndExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameEndExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameEndExpr());
             winExpr.setFrameEndExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
         if (winExpr.hasWindowFieldList()) {
             Pair<Boolean, List<Pair<Expression, Identifier>>> inlinedList =
-                    inlineUdfsInFieldList(winExpr.getWindowFieldList(), funcs);
+                    inlineUdfsInFieldList(winExpr.getWindowFieldList());
             winExpr.setWindowFieldList(inlinedList.second);
             inlined |= inlinedList.first;
         }
         if (winExpr.hasAggregateFilterExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getAggregateFilterExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getAggregateFilterExpr());
             winExpr.setAggregateFilterExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
-        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getExprList(), funcs);
+        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getExprList());
         winExpr.setExprList(inlinedList.second);
         inlined |= inlinedList.first;
         return inlined;
     }
 
     @Override
-    public Boolean visit(ListSliceExpression expression, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> expressionResult = inlineUdfsInExpr(expression.getExpr(), funcs);
+    public Boolean visit(ListSliceExpression expression, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> expressionResult = inlineUdfsInExpr(expression.getExpr());
         expression.setExpr(expressionResult.second);
         boolean inlined = expressionResult.first;
 
-        Pair<Boolean, Expression> startIndexExpressResult =
-                inlineUdfsInExpr(expression.getStartIndexExpression(), funcs);
+        Pair<Boolean, Expression> startIndexExpressResult = inlineUdfsInExpr(expression.getStartIndexExpression());
         expression.setStartIndexExpression(startIndexExpressResult.second);
         inlined |= startIndexExpressResult.first;
 
         // End index expression can be null (optional)
         if (expression.hasEndExpression()) {
-            Pair<Boolean, Expression> endIndexExpressionResult =
-                    inlineUdfsInExpr(expression.getEndIndexExpression(), funcs);
+            Pair<Boolean, Expression> endIndexExpressionResult = inlineUdfsInExpr(expression.getEndIndexExpression());
             expression.setEndIndexExpression(endIndexExpressionResult.second);
             inlined |= endIndexExpressionResult.first;
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index b5375d2..998d8e6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -257,8 +257,8 @@
 
     @Override
     public FunctionDecl visit(FunctionDecl fd, Void arg) throws CompilationException {
-        FunctionDecl copy =
-                new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        FunctionDecl copy = new FunctionDecl(fd.getSignature(), fd.getParamList(),
+                (Expression) fd.getFuncBody().accept(this, arg), fd.isStored());
         copy.setSourceLocation(fd.getSourceLocation());
         return copy;
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index faee227..b93ab87 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -384,7 +384,7 @@
     }
 
     @Override
-    public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames)
+    public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
       throws CompilationException {
         return parseImpl(new ParseFunction<FunctionDecl>() {
             @Override
@@ -399,7 +399,7 @@
                 Expression functionBodyExpr = SQLPPParser.this.FunctionBody();
                 removeCurrentScope();
                 defaultDataverse = dataverse;
-                return new FunctionDecl(signature, paramVars, functionBodyExpr);
+                return new FunctionDecl(signature, paramVars, functionBodyExpr, isStored);
             }
         });
     }
@@ -2701,7 +2701,7 @@
     for (Pair<VarIdentifier,TypeExpression> p: paramList) {
         params.add(p.getFirst());
     }
-    FunctionDecl stmt = new FunctionDecl(signature, params, funcBody);
+    FunctionDecl stmt = new FunctionDecl(signature, params, funcBody, false);
     removeCurrentScope();
     return addSourceLocation(stmt, startToken);
   }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
Gerrit-Change-Number: 10905
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905 )

Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
Gerrit-Change-Number: 10905
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 05 Apr 2021 22:45:40 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905 )


Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................

[ASTERIXDB-2871][COMP] Improve UDF recursion detection

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

Details:
- Improve UDF recursion detection
- Rewrite UDF body expression only once
- Add testcases

Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
---
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/translator/QueryTranslator.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
D asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
A asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
46 files changed, 1,393 insertions(+), 840 deletions(-)



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

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 0091304..2af2815 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
@@ -184,8 +184,9 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
+            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output,
+            boolean allowNonStoredUdfCalls, boolean inlineUdfs, Collection<VarIdentifier> externalVars,
+            IWarningCollector warningCollector) throws CompilationException {
         if (q == null) {
             return null;
         }
@@ -194,8 +195,9 @@
             generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        rw.rewrite(new ArrayList<>(declaredFunctions), q, metadataProvider,
-                new LangRewritingContext(q.getVarCounter(), warningCollector), inlineUdfs, externalVars);
+        LangRewritingContext rwCtx =
+                new LangRewritingContext(metadataProvider, declaredFunctions, warningCollector, q.getVarCounter());
+        rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfs, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index d8698f9..60a96d4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -95,13 +95,17 @@
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
+import org.apache.asterix.lang.common.literal.MissingLiteral;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
@@ -2158,16 +2162,30 @@
                     }
                 }
 
-                //Check whether the function is use-able
+                // Check whether the function is usable:
+                // create a function declaration for this function,
+                // and a query body calls this function with each argument set to 'missing'
+                FunctionDecl fd = new FunctionDecl(functionSignature, paramVars, cfs.getFunctionBodyExpression(), true);
+                fd.setSourceLocation(sourceLoc);
+                CallExpr fcall = new CallExpr(functionSignature,
+                        Collections.nCopies(paramVars.size(), new LiteralExpr(MissingLiteral.INSTANCE)));
+                fcall.setSourceLocation(sourceLoc);
                 metadataProvider.setDefaultDataverse(dv);
                 Query wrappedQuery = new Query(false);
                 wrappedQuery.setSourceLocation(sourceLoc);
-                wrappedQuery.setBody(cfs.getFunctionBodyExpression());
+                wrappedQuery.setBody(fcall);
                 wrappedQuery.setTopLevel(false);
-                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false,
-                        paramVars, warningCollector);
-                List<List<Triple<DataverseName, String, String>>> dependencies = FunctionUtil.getFunctionDependencies(
-                        rewriterFactory.createQueryRewriter(), cfs.getFunctionBodyExpression());
+                List<FunctionDecl> fdList = new ArrayList<>(declaredFunctions);
+                fdList.add(fd);
+                apiFramework.reWriteQuery(fdList, metadataProvider, wrappedQuery, sessionOutput, false, false,
+                        Collections.emptyList(), warningCollector);
+                Expression fdNormBody = fd.getNormalizedFuncBody();
+                if (fdNormBody == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                            functionSignature.toString());
+                }
+                List<List<Triple<DataverseName, String, String>>> dependencies =
+                        FunctionUtil.getFunctionDependencies(rewriterFactory.createQueryRewriter(), fdNormBody);
 
                 newInlineTypes = Collections.emptyMap();
                 function = new Function(functionSignature, paramNames, null, null, cfs.getFunctionBody(),
@@ -2864,7 +2882,7 @@
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
         Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, query, sessionOutput, true, externalVars.keySet(), warningCollector);
+                metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -2881,7 +2899,7 @@
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
         Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, insertUpsert, sessionOutput, true, externalVars.keySet(), warningCollector);
+                metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         DataverseName dataverseName = getActiveDataverseName(rewrittenInsertUpsert.getDataverseName());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8b6f05f..98f328d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -27,15 +27,17 @@
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintWriter;
+import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.IParser;
@@ -187,8 +189,9 @@
                 if (st.getKind() == Statement.Kind.QUERY) {
                     Query query = (Query) st;
                     IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
-                    rewrite(rewriter, functions, query, metadataProvider,
-                            new LangRewritingContext(query.getVarCounter(), TestUtils.NOOP_WARNING_COLLECTOR));
+                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions,
+                            TestUtils.NOOP_WARNING_COLLECTOR, query.getVarCounter());
+                    rewrite(rewriter, query, rwContext);
 
                     // Tests deep copy and deep equality.
                     Query copiedQuery = (Query) SqlppRewriteUtil.deepCopy(query);
@@ -254,28 +257,38 @@
     }
 
     // Rewrite queries.
-    // Note: we do not do inline function rewriting here because this needs real
-    // metadata access.
-    private void rewrite(IQueryRewriter rewriter, List<FunctionDecl> declaredFunctions, Query topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException {
-        PA.invokeMethod(rewriter,
-                "setup(java.util.List, org.apache.asterix.lang.common.base.IReturningStatement, "
-                        + "org.apache.asterix.metadata.declared.MetadataProvider, "
-                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext, " + "java.util.Collection)",
-                declaredFunctions, topExpr, metadataProvider, context, null);
-        PA.invokeMethod(rewriter, "resolveFunctionCalls()");
-        PA.invokeMethod(rewriter, "generateColumnNames()");
-        PA.invokeMethod(rewriter, "substituteGroupbyKeyExpression()");
-        PA.invokeMethod(rewriter, "rewriteGroupBys()");
-        PA.invokeMethod(rewriter, "rewriteSetOperations()");
-        PA.invokeMethod(rewriter, "inlineColumnAlias()");
-        PA.invokeMethod(rewriter, "rewriteWindowExpressions()");
-        PA.invokeMethod(rewriter, "rewriteGroupingSets()");
-        PA.invokeMethod(rewriter, "variableCheckAndRewrite()");
-        PA.invokeMethod(rewriter, "extractAggregatesFromCaseExpressions()");
-        PA.invokeMethod(rewriter, "rewriteGroupByAggregationSugar()");
-        PA.invokeMethod(rewriter, "rewriteWindowAggregationSugar()");
-        PA.invokeMethod(rewriter, "rewriteSpecialFunctionNames()");
+    // Note: we do not do inline function rewriting here because this needs real metadata access.
+    private void rewrite(IQueryRewriter rewriter, Query topExpr, LangRewritingContext context) throws Exception {
+        invokeMethod(rewriter, "setup", context, topExpr, null, true, false);
+        invokeMethod(rewriter, "resolveFunctionCalls");
+        invokeMethod(rewriter, "generateColumnNames");
+        invokeMethod(rewriter, "substituteGroupbyKeyExpression");
+        invokeMethod(rewriter, "rewriteGroupBys");
+        invokeMethod(rewriter, "rewriteSetOperations");
+        invokeMethod(rewriter, "inlineColumnAlias");
+        invokeMethod(rewriter, "rewriteWindowExpressions");
+        invokeMethod(rewriter, "rewriteGroupingSets");
+        invokeMethod(rewriter, "variableCheckAndRewrite");
+        invokeMethod(rewriter, "extractAggregatesFromCaseExpressions");
+        invokeMethod(rewriter, "rewriteGroupByAggregationSugar");
+        invokeMethod(rewriter, "rewriteWindowAggregationSugar");
+        invokeMethod(rewriter, "rewriteSpecialFunctionNames");
+        invokeMethod(rewriter, "rewriteOperatorExpression");
+        invokeMethod(rewriter, "rewriteCaseExpressions");
+        invokeMethod(rewriter, "rewriteListInputFunctions");
+        invokeMethod(rewriter, "rewriteRightJoins");
     }
 
+    private static void invokeMethod(Object instance, String methodName, Object... args) throws Exception {
+        PA.invokeMethod(instance, getMethodSignature(instance.getClass(), methodName), args);
+    }
+
+    private static String getMethodSignature(Class<?> cls, String methodName) throws Exception {
+        Method[] methods = cls.getDeclaredMethods();
+        Method method = Arrays.stream(methods).filter(m -> m.getName().equals(methodName)).findFirst()
+                .orElseThrow(NoSuchMethodException::new);
+        String parameterTypes =
+                Arrays.stream(method.getParameterTypes()).map(Class::getName).collect(Collectors.joining(","));
+        return String.format("%s(%s)", method.getName(), parameterTypes);
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
index 58fe98b..2e1d5a8 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
@@ -1,17 +1,15 @@
 Query:
 SELECT ELEMENT [
-CASE    LiteralExpr [TRUE]
-
-WHEN     OperatorExpr [
-      Variable [ Name=$i ]
-      >
-      Variable [ Name=$j ]
-    ]
-THEN     Variable [ Name=$i ]
-
-ELSE     Variable [ Name=$j ]
-
-END
+FunctionCall asterix.switch-case[
+  LiteralExpr [TRUE]
+  OperatorExpr [
+    Variable [ Name=$i ]
+    >
+    Variable [ Name=$j ]
+  ]
+  Variable [ Name=$i ]
+  Variable [ Name=$j ]
+]
 ]
 FROM [  OrderedListConstructor [
     LiteralExpr [LONG] [1]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
index cf95780..eeed914 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
@@ -1,15 +1,9 @@
 Query:
 SELECT ELEMENT [
-CASE    OperatorExpr [
-      LiteralExpr [LONG] [2]
-      >
-      LiteralExpr [LONG] [1]
-    ]
-
-WHEN     LiteralExpr [TRUE]
-THEN     LiteralExpr [LONG] [20]
-
-ELSE     LiteralExpr [LONG] [10]
-
-END
+FunctionCall asterix.switch-case[
+  LiteralExpr [TRUE]
+  LiteralExpr [TRUE]
+  LiteralExpr [LONG] [20]
+  LiteralExpr [LONG] [10]
+]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
index 42831c4..5fb51ac 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
@@ -4,18 +4,12 @@
   SELECT ELEMENT [
   Variable [ Name=$k ]
   ]
-  FROM [    CASE        OperatorExpr [
-          Variable [ Name=$i ]
-          >
-          Variable [ Name=$j ]
-        ]
-
-    WHEN         LiteralExpr [TRUE]
-    THEN         Variable [ Name=$i ]
-
-    ELSE         Variable [ Name=$j ]
-
-    END
+  FROM [    FunctionCall asterix.switch-case[
+      LiteralExpr [TRUE]
+      LiteralExpr [TRUE]
+      Variable [ Name=$i ]
+      Variable [ Name=$j ]
+    ]
     AS Variable [ Name=$k ]
   ]
   Where
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
index 3434965..d40210f 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
@@ -4,18 +4,16 @@
   SELECT ELEMENT [
   Variable [ Name=$k ]
   ]
-  FROM [    CASE        LiteralExpr [TRUE]
-
-    WHEN         OperatorExpr [
-          Variable [ Name=$i ]
-          >
-          Variable [ Name=$j ]
-        ]
-    THEN         Variable [ Name=$i ]
-
-    ELSE         Variable [ Name=$j ]
-
-    END
+  FROM [    FunctionCall asterix.switch-case[
+      LiteralExpr [TRUE]
+      OperatorExpr [
+        Variable [ Name=$i ]
+        >
+        Variable [ Name=$j ]
+      ]
+      Variable [ Name=$i ]
+      Variable [ Name=$j ]
+    ]
     AS Variable [ Name=$k ]
   ]
   Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
new file mode 100644
index 0000000..cf1603e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test illegal recursive function calls
+ */
+drop dataverse test if exists;
+create dataverse test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
new file mode 100644
index 0000000..ee465fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f2a(a) { - f2a(a) };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
new file mode 100644
index 0000000..79fc6bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f3a(a) { -a };
+create or replace function f3a(a) { f3a(a) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
new file mode 100644
index 0000000..24af0d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+create function f4a(a) { -a };
+create function f4b(b) { f4a(b) };
+create or replace function f4a(a) { f4b(a) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
new file mode 100644
index 0000000..c6ded74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f5a(a) { - f5a(a) };
+
+f5a(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
new file mode 100644
index 0000000..f47b781
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f6a(a) { f6b(a) + f6b(-a) };
+declare function f6b(b) { f6a(-b) + f6a(b) };
+
+f6a(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
new file mode 100644
index 0000000..0c89f14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f7a(a) { f7b(a) + f7b(-a) };
+declare function f7b(b) { f7a(-b) + f7a(b) };
+declare function f7c(c) { f7b(c) };
+
+f7c(1);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
new file mode 100644
index 0000000..bf5ca06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in CREATE FUNCTION (varargs)
+ * Expected Res : Failure
+ */
+
+use test;
+
+create function f8a(...) { - args[0] - args[1] };
+
+create function f8b(...) { f8a(args[0], args[1]) + f8a(args[1], args[0]) };
+
+create or replace function f8a(...) { f8b(args[0], args[1]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
new file mode 100644
index 0000000..e5d4e98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Recursion in DECLARE FUNCTION (varargs)
+ * Expected Res : Failure
+ */
+use test;
+
+declare function f9a(...) { f9b(args[0]) + f9b(-args[1]) };
+
+declare function f9b(...) { f9a(-args[0]) + f9a(args[1]) };
+
+declare function f9c(...) { f9b(args[0], args[1]) };
+
+f9c(1, 2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
new file mode 100644
index 0000000..f237df5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function f1a(a) { -a };
+
+create function f1b(b) { f1a(b) + f1a(b) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
new file mode 100644
index 0000000..2c4236b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f1a(r) f1a, f1b(r) f1b
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab206d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+create function f3a(a) { -a };
+
+create function f3b(b) { f3a(b) };
+
+create function f3c(c) { f3a(c) + f3b(c) };
+
+create function f3d(d) { f3a(d) + f3b(d) + f3c(d) };
+
+create function f3e(e) { f3a(e) + f3b(e) + f3c(e) + f3d(e) };
+
+create function f3f(f) { f3a(f) + f3b(f) + f3c(f) + f3d(f) + f3e(f) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
new file mode 100644
index 0000000..2ad1f20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f3a(r) f3a, f3b(r) f3b, f3c(r) f3c, f3d(r) f3d, f3e(r) f3e, f3f(r) f3f
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
new file mode 100644
index 0000000..e4ce3f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
@@ -0,0 +1,37 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion (vararg functions)
+ */
+
+use test;
+
+create function f5a(...) { - args[0] - args[1] };
+
+create function f5b(...) { f5a(args[0], args[1]) };
+
+create function f5c(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) };
+
+create function f5d(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) };
+
+create function f5e(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) + f5d(args[0], args[1]) };
+
+create function f5f(...) { f5a(args[0], args[1]) + f5b(args[0], args[1]) + f5c(args[0], args[1]) + f5d(args[0], args[1]) + f5e(args[0], args[1]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
new file mode 100644
index 0000000..5e8087b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : No recursion
+ */
+
+use test;
+
+select r, f5a(r, r+1) f5a, f5b(r, r+1) f5b, f5c(r, r+1) f5c, f5d(r, r+1) f5d, f5e(r, r+1) f5e, f5f(r, r+1) f5f
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
new file mode 100644
index 0000000..b7e7b81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description  : Cannot call a declared function from
+ *                a stored function
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+declare function f1a(a) { -a };
+
+create function f1b(b) { f1a(b) + f1a(b) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
new file mode 100644
index 0000000..0ebb577
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Cannot call a declared function from
+ *                a stored function (varargs)
+ */
+
+use test;
+
+declare function f2a(...) { - args[0] - args[1] };
+
+create function f2b(...) { f2a(args[0], args[1]) + f2a(args[1], args[0]) };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
new file mode 100644
index 0000000..c82ed63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f1a": -1, "f1b": -2 }
+{ "r": 2, "f1a": -2, "f1b": -4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
new file mode 100644
index 0000000..5c663b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f3a": -1, "f3b": -1, "f3c": -2, "f3d": -4, "f3e": -8, "f3f": -16 }
+{ "r": 2, "f3a": -2, "f3b": -2, "f3c": -4, "f3d": -8, "f3e": -16, "f3f": -32 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
new file mode 100644
index 0000000..6389c0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "f5a": -3, "f5b": -3, "f5c": -6, "f5d": -12, "f5e": -24, "f5f": -48 }
+{ "r": 2, "f5a": -5, "f5b": -5, "f5c": -10, "f5d": -20, "f5e": -40, "f5f": -80 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 19e0965..04a274a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -12341,6 +12341,31 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf37_recursion">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1149: Illegal function recursion (in line 24, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 25, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 26, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 27, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 28, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function recursion (in line 31, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf38_no_recursion">
+        <output-dir compare="Text">udf38_no_recursion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf39_illegal_call">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1150: Illegal use of function test.f1a(1) (in line 32, at column 26)</expected-error>
+        <expected-error>ASX1150: Illegal use of function test.f2a(...) (in line 29, at column 28)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="f01">
         <output-dir compare="Text">f01</output-dir>
         <expected-error>ASX1081: Cannot find function with signature test.tinyint()</expected-error>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 47a388d..210ce0f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -233,6 +233,8 @@
     UNKNOWN_FEED_POLICY(1146),
     CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS(1147),
     CANNOT_DROP_OBJECT_DEPENDENT_EXISTS(1148),
+    ILLEGAL_FUNCTION_RECURSION(1149),
+    ILLEGAL_FUNCTION_USE(1150),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 8954766..ab65a46 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -235,6 +235,8 @@
 1146 = Cannot find feed policy with name %1$s
 1147 = Cannot drop dataverse: %1$s %2$s being used by %3$s %4$s
 1148 = Cannot drop %1$s %2$s being used by %3$s %4$s
+1149 = Illegal function recursion
+1150 = Illegal use of function %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
index 0fe58ff..3df490c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
@@ -35,7 +35,8 @@
 
     List<String> parseMultipartIdentifier() throws CompilationException;
 
-    FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames) throws CompilationException;
+    FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
+            throws CompilationException;
 
     /**
      * Gets the warnings generated during parsing
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 3e1851c..f28de55 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -19,40 +19,37 @@
 package org.apache.asterix.lang.common.base;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 
 public interface IQueryRewriter {
 
     /**
      * Rewrite a query at the AST level.
-     * @param declaredFunctions,
-     *          a list of declared functions associated with the query.
      * @param topExpr,
      *          the query to be rewritten.
-     * @param metadataProvider,
-     *          providing the definition of created (i.e., stored) user-defined functions.
      * @param context
      *          rewriting context
+     * @param allowNonStoredUdfCalls
+     *          whether calls to non-stored user-defined functions should be resolved
+     * @param inlineUdfs
+     *          whether user defined functions should be inlines
      * @param externalVars
-     *          external variables
+     *          statement parameters (external variables)
      */
-    void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context, boolean inlineUdfs, Collection<VarIdentifier> externalVars)
-            throws CompilationException;
+    void rewrite(LangRewritingContext context, IReturningStatement topExpr, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException;
 
     /**
      * Find the function calls used by a given expression
      */
-    Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException;
+    void getFunctionCalls(Expression expression, Collection<? super AbstractCallExpression> outCalls)
+            throws CompilationException;
 
     /**
      * Find all external variables (positional and named variables) in given expression
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
deleted file mode 100644
index d18aa86..0000000
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
+++ /dev/null
@@ -1,62 +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.lang.common.parser;
-
-import java.io.StringReader;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.metadata.entities.Function;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-
-public class FunctionParser {
-
-    private final IParserFactory parserFactory;
-
-    public FunctionParser(IParserFactory parserFactory) {
-        this.parserFactory = parserFactory;
-    }
-
-    public String getLanguage() {
-        return parserFactory.getLanguage();
-    }
-
-    public FunctionDecl getFunctionDecl(Function function, IWarningCollector warningCollector)
-            throws CompilationException {
-        if (!function.getLanguage().equals(getLanguage())) {
-            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE, getLanguage(),
-                    function.getLanguage());
-        }
-        IParser parser = parserFactory.createParser(new StringReader(function.getFunctionBody()));
-        try {
-            FunctionDecl functionDecl = parser.parseFunctionBody(function.getSignature(), function.getParameterNames());
-            if (warningCollector != null) {
-                parser.getWarnings(warningCollector);
-            }
-            return functionDecl;
-        } catch (CompilationException e) {
-            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, function.getSignature(),
-                    e.getMessage());
-        }
-    }
-}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 563e07e..675d0d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -19,21 +19,31 @@
 package org.apache.asterix.lang.common.rewrites;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
 public final class LangRewritingContext {
+    private final MetadataProvider metadataProvider;
     private final IWarningCollector warningCollector;
-    private Counter varCounter;
+    private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
+    private final Counter varCounter;
     private int systemVarCounter = 1;
-    private Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
+    private final Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
 
-    public LangRewritingContext(int varCounter, IWarningCollector warningCollector) {
-        this.varCounter = new Counter(varCounter);
+    public LangRewritingContext(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions,
+            IWarningCollector warningCollector, int varCounter) {
+        this.metadataProvider = metadataProvider;
         this.warningCollector = warningCollector;
+        this.declaredFunctions = FunctionUtil.getFunctionMap(declaredFunctions);
+        this.varCounter = new Counter(varCounter);
     }
 
     public Counter getVarCounter() {
@@ -75,4 +85,12 @@
     public IWarningCollector getWarningCollector() {
         return warningCollector;
     }
+
+    public MetadataProvider getMetadataProvider() {
+        return metadataProvider;
+    }
+
+    public Map<FunctionSignature, FunctionDecl> getDeclaredFunctions() {
+        return declaredFunctions;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
index 4161824..2ef11ad 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
@@ -29,14 +29,18 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class FunctionDecl extends AbstractStatement {
-    private FunctionSignature signature;
-    private List<VarIdentifier> paramList;
+    private final FunctionSignature signature;
+    private final List<VarIdentifier> paramList;
     private Expression funcBody;
+    private Expression funcBodyNormalized;
+    private final boolean isStored;
 
-    public FunctionDecl(FunctionSignature signature, List<VarIdentifier> paramList, Expression funcBody) {
+    public FunctionDecl(FunctionSignature signature, List<VarIdentifier> paramList, Expression funcBody,
+            boolean isStored) {
         this.signature = signature;
         this.paramList = paramList;
         this.funcBody = funcBody;
+        this.isStored = isStored;
     }
 
     public FunctionSignature getSignature() {
@@ -53,14 +57,19 @@
 
     public void setFuncBody(Expression funcBody) {
         this.funcBody = funcBody;
+        this.funcBodyNormalized = null;
     }
 
-    public void setSignature(FunctionSignature signature) {
-        this.signature = signature;
+    public Expression getNormalizedFuncBody() {
+        return funcBodyNormalized;
     }
 
-    public void setParamList(List<VarIdentifier> paramList) {
-        this.paramList = paramList;
+    public void setNormalizedFuncBody(Expression funcBody) {
+        this.funcBodyNormalized = funcBody;
+    }
+
+    public boolean isStored() {
+        return isStored;
     }
 
     @Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 7f4e078..3a6bf97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -19,11 +19,14 @@
 
 package org.apache.asterix.lang.common.util;
 
+import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.function.BiFunction;
@@ -34,6 +37,8 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -41,8 +46,8 @@
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
-import org.apache.asterix.lang.common.parser.FunctionParser;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.BuiltinTypeMap;
 import org.apache.asterix.metadata.entities.Dataverse;
@@ -62,6 +67,10 @@
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
+import com.google.common.graph.GraphBuilder;
+import com.google.common.graph.Graphs;
+import com.google.common.graph.MutableGraph;
+
 public class FunctionUtil {
 
     public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
@@ -104,54 +113,50 @@
         }
     }
 
-    @FunctionalInterface
-    public interface IFunctionCollector {
-        Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException;
-    }
-
     public static FunctionSignature resolveFunctionCall(FunctionSignature fs, SourceLocation sourceLoc,
-            MetadataProvider metadataProvider, Set<FunctionSignature> declaredFunctions,
-            BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver) throws CompilationException {
-        int arity = fs.getArity();
+            MetadataProvider metadataProvider, BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver,
+            boolean searchUdfs, Map<FunctionSignature, FunctionDecl> declaredFunctionMap,
+            boolean allowNonStoredUdfCalls) throws CompilationException {
         DataverseName dataverse = fs.getDataverseName();
         if (dataverse == null) {
             dataverse = metadataProvider.getDefaultDataverseName();
         }
-        boolean isBuiltinFuncDataverse =
-                dataverse.equals(FunctionConstants.ASTERIX_DV) || dataverse.equals(FunctionConstants.ALGEBRICKS_DV);
-
-        if (!isBuiltinFuncDataverse) {
+        if (searchUdfs && !isBuiltinFunctionDataverse(dataverse)) {
             // attempt to resolve to a user-defined function
             FunctionSignature fsWithDv =
-                    fs.getDataverseName() == null ? new FunctionSignature(dataverse, fs.getName(), arity) : fs;
-            if (declaredFunctions.contains(fsWithDv)) {
-                return fsWithDv;
-            }
+                    fs.getDataverseName() == null ? new FunctionSignature(dataverse, fs.getName(), fs.getArity()) : fs;
             FunctionSignature fsWithDvVarargs =
                     new FunctionSignature(fsWithDv.getDataverseName(), fsWithDv.getName(), FunctionIdentifier.VARARGS);
-            if (declaredFunctions.contains(fsWithDvVarargs)) {
-                return fsWithDvVarargs;
+
+            FunctionDecl fd = declaredFunctionMap.get(fsWithDv);
+            if (fd == null) {
+                fd = declaredFunctionMap.get(fsWithDvVarargs);
+            }
+            if (fd != null) {
+                if (!allowNonStoredUdfCalls && !fd.isStored()) {
+                    throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_USE, sourceLoc,
+                            fd.getSignature().toString());
+                }
+                return fd.getSignature();
             }
             try {
-                Function function = metadataProvider.lookupUserDefinedFunction(fsWithDv);
-                if (function != null) {
-                    return fsWithDv;
+                Function fn = metadataProvider.lookupUserDefinedFunction(fsWithDv);
+                if (fn == null) {
+                    fn = metadataProvider.lookupUserDefinedFunction(fsWithDvVarargs);
                 }
-                function = metadataProvider.lookupUserDefinedFunction(fsWithDvVarargs);
-                if (function != null) {
-                    return fsWithDvVarargs;
+                if (fn != null) {
+                    return fn.getSignature();
                 }
             } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
+                throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, e, sourceLoc, fs.toString());
             }
-
             // fail if the dataverse was specified in the function call but this dataverse does not exist
             if (fs.getDataverseName() != null) {
                 Dataverse dv;
                 try {
                     dv = metadataProvider.findDataverse(dataverse);
                 } catch (AlgebricksException e) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, dataverse);
                 }
                 if (dv == null) {
                     throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverse);
@@ -165,13 +170,22 @@
         if (mappedName != null) {
             name = mappedName;
         }
-        FunctionSignature fsBuiltin = builtinFunctionResolver.apply(name, arity);
+        FunctionSignature fsBuiltin = builtinFunctionResolver.apply(name, fs.getArity());
         if (fsBuiltin == null) {
             throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, fs.toString());
         }
         return fsBuiltin;
     }
 
+    public static boolean isBuiltinFunctionSignature(FunctionSignature fs) {
+        return isBuiltinFunctionDataverse(Objects.requireNonNull(fs.getDataverseName()))
+                || BuiltinFunctions.getBuiltinFunctionInfo(fs.createFunctionIdentifier()) != null;
+    }
+
+    private static boolean isBuiltinFunctionDataverse(DataverseName dataverse) {
+        return FunctionConstants.ASTERIX_DV.equals(dataverse) || FunctionConstants.ALGEBRICKS_DV.equals(dataverse);
+    }
+
     public static BiFunction<String, Integer, FunctionSignature> createBuiltinFunctionResolver(
             MetadataProvider metadataProvider) {
         boolean includePrivateFunctions = getImportPrivateFunctions(metadataProvider);
@@ -193,88 +207,38 @@
         };
     }
 
-    /**
-     * Retrieve stored functions (from CREATE FUNCTION statements) that have been
-     * used in an expression.
-     *
-     * @param metadataProvider,
-     *            the metadata provider
-     * @param expression,
-     *            the expression for analysis
-     * @param declaredFunctions,
-     *            a set of declared functions in the query, which can potentially
-     *            override stored functions.
-     * @param functionCollector,
-     *            for collecting function calls in the <code>expression</code>
-     * @param functionParser,
-     *            for parsing stored functions in the string represetnation.
-     * @param warningCollector
-     *            for reporting warnings encountered during parsing
-     * @throws CompilationException
-     */
-    public static List<FunctionDecl> retrieveUsedStoredFunctions(MetadataProvider metadataProvider,
-            Expression expression, List<FunctionSignature> declaredFunctions, List<FunctionDecl> inputFunctionDecls,
-            IFunctionCollector functionCollector, FunctionParser functionParser, IWarningCollector warningCollector)
-            throws CompilationException {
-        if (expression == null) {
-            return Collections.emptyList();
-        }
-        List<FunctionDecl> functionDecls =
-                inputFunctionDecls == null ? new ArrayList<>() : new ArrayList<>(inputFunctionDecls);
-        Set<AbstractCallExpression> functionCalls = functionCollector.getFunctionCalls(expression);
-        Set<FunctionSignature> functionSignatures = new HashSet<>();
-        for (AbstractCallExpression functionCall : functionCalls) {
-            switch (functionCall.getKind()) {
-                case CALL_EXPRESSION:
-                    FunctionSignature fs = functionCall.getFunctionSignature();
-                    if (fs.getDataverseName() == null) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
-                                functionCall.getSourceLocation(), fs);
+    public static void checkFunctionRecursion(Map<FunctionSignature, FunctionDecl> functionDeclMap,
+            java.util.function.Function<Collection<AbstractCallExpression>, GatherFunctionCallsVisitor> gfcFactory,
+            SourceLocation sourceLoc) throws CompilationException {
+        List<AbstractCallExpression> callList = new ArrayList<>();
+        GatherFunctionCallsVisitor gfc = gfcFactory.apply(callList);
+        MutableGraph<FunctionDecl> graph = GraphBuilder.directed().allowsSelfLoops(true).build();
+        for (FunctionDecl fdFrom : functionDeclMap.values()) {
+            callList.clear();
+            fdFrom.getNormalizedFuncBody().accept(gfc, null);
+            for (AbstractCallExpression callExpr : callList) {
+                if (callExpr.getKind() == Expression.Kind.CALL_EXPRESSION) {
+                    FunctionSignature callSignature = callExpr.getFunctionSignature();
+                    FunctionDecl fdTo = functionDeclMap.get(callSignature);
+                    if (fdTo != null) {
+                        graph.putEdge(fdFrom, fdTo);
                     }
-                    if (!functionSignatures.add(fs)) {
-                        // already seen this signature
-                        continue;
-                    }
-                    if (declaredFunctions != null && declaredFunctions.contains(fs)) {
-                        continue;
-                    }
-                    Function function;
-                    try {
-                        function = metadataProvider.lookupUserDefinedFunction(fs);
-                    } catch (AlgebricksException e) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, functionCall.getSourceLocation(),
-                                e.toString());
-                    }
-                    if (function == null || !functionParser.getLanguage().equals(function.getLanguage())) {
-                        // the function is either unknown, builtin, or in a different language.
-                        // either way we ignore it here because it will be handled by the function inlining rule later
-                        continue;
-                    }
-
-                    FunctionDecl functionDecl = functionParser.getFunctionDecl(function, warningCollector);
-                    if (functionDecls.contains(functionDecl)) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
-                                "Recursive invocation " + functionDecls.get(functionDecls.size() - 1).getSignature()
-                                        + " <==> " + functionDecl.getSignature());
-                    }
-                    functionDecls.add(functionDecl);
-                    functionDecls = retrieveUsedStoredFunctions(metadataProvider, functionDecl.getFuncBody(),
-                            declaredFunctions, functionDecls, functionCollector, functionParser, warningCollector);
-                    break;
-                case WINDOW_EXPRESSION:
-                    // there cannot be used-defined window functions
-                    break;
-                default:
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expression.getSourceLocation(),
-                            functionCall.getFunctionSignature().toString(false));
+                }
             }
         }
-        return functionDecls;
+        if (Graphs.hasCycle(graph)) {
+            throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_RECURSION, sourceLoc);
+        }
     }
 
     public static List<List<Triple<DataverseName, String, String>>> getFunctionDependencies(IQueryRewriter rewriter,
             Expression expression) throws CompilationException {
-        Set<AbstractCallExpression> functionCalls = rewriter.getFunctionCalls(expression);
+        List<AbstractCallExpression> functionCalls = new ArrayList<>();
+        rewriter.getFunctionCalls(expression, functionCalls);
+        // Duplicate elimination
+        Set<FunctionSignature> seenFunctions = new HashSet<>();
+        Set<Pair<DataverseName, String>> seenDatasets = new HashSet<>();
+        Set<Pair<DataverseName, String>> seenSynonyms = new HashSet<>();
         //Get the List of used functions and used datasets
         List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
         List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
@@ -289,17 +253,23 @@
                         if (callExpr.getExprList().size() > 2) {
                             // resolved via synonym -> store synonym name as a dependency
                             Pair<DataverseName, String> synonymReference = parseDatasetFunctionArguments(callExpr, 2);
-                            synonymDependencies
-                                    .add(new Triple<>(synonymReference.first, synonymReference.second, null));
+                            if (seenSynonyms.add(synonymReference)) {
+                                synonymDependencies
+                                        .add(new Triple<>(synonymReference.first, synonymReference.second, null));
+                            }
                         } else {
                             // resolved directly -> store dataset name as a dependency
                             Pair<DataverseName, String> datasetReference = parseDatasetFunctionArguments(callExpr, 0);
-                            datasetDependencies
-                                    .add(new Triple<>(datasetReference.first, datasetReference.second, null));
+                            if (seenDatasets.add(datasetReference)) {
+                                datasetDependencies
+                                        .add(new Triple<>(datasetReference.first, datasetReference.second, null));
+                            }
                         }
                     } else if (BuiltinFunctions.getBuiltinFunctionInfo(signature.createFunctionIdentifier()) == null) {
-                        functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
-                                Integer.toString(signature.getArity())));
+                        if (seenFunctions.add(signature)) {
+                            functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
+                                    Integer.toString(signature.getArity())));
+                        }
                     }
                     break;
                 case WINDOW_EXPRESSION:
@@ -374,14 +344,35 @@
         return (value != null) && Boolean.parseBoolean(value.toLowerCase());
     }
 
-    public static Set<FunctionSignature> getFunctionSignatures(List<FunctionDecl> declaredFunctions) {
+    public static Map<FunctionSignature, FunctionDecl> getFunctionMap(List<FunctionDecl> declaredFunctions) {
         if (declaredFunctions == null || declaredFunctions.isEmpty()) {
-            return Collections.emptySet();
+            return Collections.emptyMap();
         }
-        Set<FunctionSignature> result = new HashSet<>();
+        Map<FunctionSignature, FunctionDecl> result = new HashMap<>();
         for (FunctionDecl fd : declaredFunctions) {
-            result.add(fd.getSignature());
+            result.put(fd.getSignature(), fd);
         }
         return result;
     }
+
+    public static FunctionDecl parseStoredFunction(Function function, IParserFactory parserFactory,
+            IWarningCollector warningCollector, SourceLocation sourceLoc) throws CompilationException {
+        if (!function.getLanguage().equals(parserFactory.getLanguage())) {
+            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE, sourceLoc,
+                    parserFactory.getLanguage(), function.getLanguage());
+        }
+        IParser parser = parserFactory.createParser(new StringReader(function.getFunctionBody()));
+        try {
+            FunctionDecl functionDecl =
+                    parser.parseFunctionBody(function.getSignature(), function.getParameterNames(), true);
+            functionDecl.setSourceLocation(sourceLoc);
+            if (warningCollector != null) {
+                parser.getWarnings(warningCollector);
+            }
+            return functionDecl;
+        } catch (CompilationException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, sourceLoc,
+                    function.getSignature(), e.getMessage());
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index f1bcb93..cba6bb5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -26,12 +26,9 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.base.IQueryRewriter;
-import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -58,31 +55,26 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-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;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, List<FunctionDecl>> {
+public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, Void> {
 
     protected final LangRewritingContext context;
-    protected final CloneAndSubstituteVariablesVisitor cloneVisitor;
-    private final IRewriterFactory rewriterFactory;
-    private final List<FunctionDecl> declaredFunctions;
-    private final MetadataProvider metadataProvider;
 
-    public AbstractInlineUdfsVisitor(LangRewritingContext context, IRewriterFactory rewriterFactory,
-            List<FunctionDecl> declaredFunctions, MetadataProvider metadataProvider,
+    protected final Map<FunctionSignature, FunctionDecl> usedUDFs;
+
+    protected final CloneAndSubstituteVariablesVisitor cloneVisitor;
+
+    public AbstractInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs,
             CloneAndSubstituteVariablesVisitor cloneVisitor) {
         this.context = context;
+        this.usedUDFs = usedUDFs;
         this.cloneVisitor = cloneVisitor;
-        this.rewriterFactory = rewriterFactory;
-        this.declaredFunctions = declaredFunctions;
-        this.metadataProvider = metadataProvider;
     }
 
     /**
@@ -96,36 +88,33 @@
             throws CompilationException;
 
     @Override
-    public Boolean visit(Query q, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(q.getBody(), arg);
+    public Boolean visit(Query q, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(q.getBody());
         q.setBody(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(FunctionDecl fd, List<FunctionDecl> arg) throws CompilationException {
-        // Careful, we should only do this after analyzing the graph of function
-        // calls.
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fd.getFuncBody(), arg);
-        fd.setFuncBody(p.second);
-        return p.first;
+    public Boolean visit(FunctionDecl fd, Void arg) throws CompilationException {
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fd.getSourceLocation(),
+                fd.getSignature().toString());
     }
 
     @Override
-    public Boolean visit(ListConstructor lc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(lc.getExprList(), arg);
+    public Boolean visit(ListConstructor lc, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(lc.getExprList());
         lc.setExprList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(RecordConstructor rc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(RecordConstructor rc, Void arg) throws CompilationException {
         boolean changed = false;
         for (FieldBinding b : rc.getFbList()) {
-            Pair<Boolean, Expression> leftExprInlined = inlineUdfsInExpr(b.getLeftExpr(), arg);
+            Pair<Boolean, Expression> leftExprInlined = inlineUdfsInExpr(b.getLeftExpr());
             b.setLeftExpr(leftExprInlined.second);
             changed = changed || leftExprInlined.first;
-            Pair<Boolean, Expression> rightExprInlined = inlineUdfsInExpr(b.getRightExpr(), arg);
+            Pair<Boolean, Expression> rightExprInlined = inlineUdfsInExpr(b.getRightExpr());
             b.setRightExpr(rightExprInlined.second);
             changed = changed || rightExprInlined.first;
         }
@@ -133,12 +122,12 @@
     }
 
     @Override
-    public Boolean visit(CallExpr callExpr, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(callExpr.getExprList(), arg);
+    public Boolean visit(CallExpr callExpr, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(callExpr.getExprList());
         callExpr.setExprList(p.second);
         boolean changed = p.first;
         if (callExpr.hasAggregateFilterExpr()) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(callExpr.getAggregateFilterExpr(), arg);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(callExpr.getAggregateFilterExpr());
             callExpr.setAggregateFilterExpr(be.second);
             changed |= be.first;
         }
@@ -146,96 +135,96 @@
     }
 
     @Override
-    public Boolean visit(OperatorExpr ifbo, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(ifbo.getExprList(), arg);
+    public Boolean visit(OperatorExpr ifbo, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(ifbo.getExprList());
         ifbo.setExprList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(FieldAccessor fa, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr(), arg);
+    public Boolean visit(FieldAccessor fa, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
         fa.setExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(IndexAccessor fa, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr(), arg);
+    public Boolean visit(IndexAccessor fa, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
         fa.setExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(IfExpr ifexpr, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(ifexpr.getCondExpr(), arg);
+    public Boolean visit(IfExpr ifexpr, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(ifexpr.getCondExpr());
         ifexpr.setCondExpr(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(ifexpr.getThenExpr(), arg);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(ifexpr.getThenExpr());
         ifexpr.setThenExpr(p2.second);
-        Pair<Boolean, Expression> p3 = inlineUdfsInExpr(ifexpr.getElseExpr(), arg);
+        Pair<Boolean, Expression> p3 = inlineUdfsInExpr(ifexpr.getElseExpr());
         ifexpr.setElseExpr(p3.second);
         return p1.first || p2.first || p3.first;
     }
 
     @Override
-    public Boolean visit(QuantifiedExpression qe, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(QuantifiedExpression qe, Void arg) throws CompilationException {
         boolean changed = false;
         for (QuantifiedPair t : qe.getQuantifiedList()) {
-            Pair<Boolean, Expression> p = inlineUdfsInExpr(t.getExpr(), arg);
+            Pair<Boolean, Expression> p = inlineUdfsInExpr(t.getExpr());
             t.setExpr(p.second);
             if (p.first) {
                 changed = true;
             }
         }
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(qe.getSatisfiesExpr(), arg);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(qe.getSatisfiesExpr());
         qe.setSatisfiesExpr(p2.second);
         return changed || p2.first;
     }
 
     @Override
-    public Boolean visit(LetClause lc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(lc.getBindingExpr(), arg);
+    public Boolean visit(LetClause lc, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(lc.getBindingExpr());
         lc.setBindingExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(WhereClause wc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(wc.getWhereExpr(), arg);
+    public Boolean visit(WhereClause wc, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(wc.getWhereExpr());
         wc.setWhereExpr(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(OrderbyClause oc, List<FunctionDecl> arg) throws CompilationException {
-        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(oc.getOrderbyList(), arg);
+    public Boolean visit(OrderbyClause oc, Void arg) throws CompilationException {
+        Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(oc.getOrderbyList());
         oc.setOrderbyList(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(GroupbyClause gc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(GroupbyClause gc, Void arg) throws CompilationException {
         boolean changed = false;
         List<List<GbyVariableExpressionPair>> gbyList = gc.getGbyPairList();
         List<List<GbyVariableExpressionPair>> newGbyList = new ArrayList<>(gbyList.size());
         for (List<GbyVariableExpressionPair> gbyPairList : gbyList) {
-            Pair<Boolean, List<GbyVariableExpressionPair>> p1 = inlineUdfsInGbyPairList(gbyPairList, arg);
+            Pair<Boolean, List<GbyVariableExpressionPair>> p1 = inlineUdfsInGbyPairList(gbyPairList);
             newGbyList.add(p1.second);
             changed |= p1.first;
         }
         gc.setGbyPairList(newGbyList);
         if (gc.hasDecorList()) {
-            Pair<Boolean, List<GbyVariableExpressionPair>> p2 = inlineUdfsInGbyPairList(gc.getDecorPairList(), arg);
+            Pair<Boolean, List<GbyVariableExpressionPair>> p2 = inlineUdfsInGbyPairList(gc.getDecorPairList());
             gc.setDecorPairList(p2.second);
             changed |= p2.first;
         }
         if (gc.hasGroupFieldList()) {
-            Pair<Boolean, List<Pair<Expression, Identifier>>> p3 = inlineUdfsInFieldList(gc.getGroupFieldList(), arg);
+            Pair<Boolean, List<Pair<Expression, Identifier>>> p3 = inlineUdfsInFieldList(gc.getGroupFieldList());
             gc.setGroupFieldList(p3.second);
             changed |= p3.first;
         }
         if (gc.hasWithMap()) {
-            Pair<Boolean, Map<Expression, VariableExpr>> p4 = inlineUdfsInVarMap(gc.getWithVarMap(), arg);
+            Pair<Boolean, Map<Expression, VariableExpr>> p4 = inlineUdfsInVarMap(gc.getWithVarMap());
             gc.setWithVarMap(p4.second);
             changed |= p4.first;
         }
@@ -243,15 +232,15 @@
     }
 
     @Override
-    public Boolean visit(LimitClause lc, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(LimitClause lc, Void arg) throws CompilationException {
         boolean changed = false;
         if (lc.hasLimitExpr()) {
-            Pair<Boolean, Expression> p1 = inlineUdfsInExpr(lc.getLimitExpr(), arg);
+            Pair<Boolean, Expression> p1 = inlineUdfsInExpr(lc.getLimitExpr());
             lc.setLimitExpr(p1.second);
             changed = p1.first;
         }
         if (lc.hasOffset()) {
-            Pair<Boolean, Expression> p2 = inlineUdfsInExpr(lc.getOffset(), arg);
+            Pair<Boolean, Expression> p2 = inlineUdfsInExpr(lc.getOffset());
             lc.setOffset(p2.second);
             changed |= p2.first;
         }
@@ -259,131 +248,132 @@
     }
 
     @Override
-    public Boolean visit(UnaryExpr u, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(UnaryExpr u, Void arg) throws CompilationException {
         return u.getExpr().accept(this, arg);
     }
 
     @Override
-    public Boolean visit(VariableExpr v, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(VariableExpr v, Void arg) throws CompilationException {
         return false;
     }
 
     @Override
-    public Boolean visit(LiteralExpr l, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(LiteralExpr l, Void arg) throws CompilationException {
         return false;
     }
 
     @Override
-    public Boolean visit(InsertStatement insert, List<FunctionDecl> arg) throws CompilationException {
+    public Boolean visit(InsertStatement insert, Void arg) throws CompilationException {
         boolean changed = false;
         Expression returnExpression = insert.getReturnExpression();
         if (returnExpression != null) {
-            Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression, arg);
+            Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression);
             insert.setReturnExpression(rewrittenReturnExpr.second);
             changed |= rewrittenReturnExpr.first;
         }
-        Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody(), arg);
+        Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody());
         insert.setBody(rewrittenBodyExpression.second);
         return changed || rewrittenBodyExpression.first;
     }
 
-    protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr, List<FunctionDecl> arg)
-            throws CompilationException {
+    protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr) throws CompilationException {
         if (expr.getKind() != Kind.CALL_EXPRESSION) {
-            boolean r = expr.accept(this, arg);
+            boolean r = expr.accept(this, null);
             return new Pair<>(r, expr);
         }
         CallExpr f = (CallExpr) expr;
-        boolean r = expr.accept(this, arg);
-        FunctionDecl implem = findFuncDeclaration(f.getFunctionSignature(), arg);
-        if (implem == null) {
+        boolean r = expr.accept(this, null);
+        FunctionSignature fs = f.getFunctionSignature();
+        if (FunctionUtil.isBuiltinFunctionSignature(fs)) {
             return new Pair<>(r, expr);
-        } else {
-            if (f.hasAggregateFilterExpr()) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_FILTER_CLAUSE,
-                        f.getSourceLocation());
-            }
-            // Rewrite the function body itself (without setting unbounded variables to dataset access).
-            // TODO(buyingyi): throw an exception for recursive function definition or limit the stack depth.
-            implem.setFuncBody(rewriteFunctionBody(implem));
-            // it's one of the functions we want to inline
-            List<Expression> argList = f.getExprList();
-            int argCount = argList.size();
-            List<LetClause> clauses = new ArrayList<>(argCount + 1);
-            List<Expression> argVars = new ArrayList<>(argCount);
-            for (Expression e : f.getExprList()) {
-                // Obs: we could do smth about passing also literals, or let
-                // variable inlining to take care of this.
-                VarIdentifier argVar;
-                if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
-                    argVar = ((VariableExpr) e).getVar();
-                } else {
-                    SourceLocation sourceLoc = e.getSourceLocation();
-                    argVar = context.newVariable();
-                    Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
-                            e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
-                    VariableExpr newVRef1 = new VariableExpr(argVar);
-                    newVRef1.setSourceLocation(sourceLoc);
-                    LetClause c = new LetClause(newVRef1, (Expression) p1.first);
-                    c.setSourceLocation(sourceLoc);
-                    clauses.add(c);
-                }
-
-                VariableExpr argVarExpr = new VariableExpr(argVar);
-                argVarExpr.setSourceLocation(e.getSourceLocation());
-                argVars.add(argVarExpr);
-            }
-
-            VariableSubstitutionEnvironment subst = new VariableSubstitutionEnvironment();
-            List<VarIdentifier> paramList = implem.getParamList();
-            if (implem.getSignature().getArity() == FunctionIdentifier.VARARGS) {
-                if (paramList.size() != 1) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
-                            paramList.size());
-                }
-                VarIdentifier paramVarargs = paramList.get(0);
-                CallExpr argsListExpr =
-                        new CallExpr(new FunctionSignature(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR), argVars);
-                argsListExpr.setSourceLocation(expr.getSourceLocation());
-
-                VarIdentifier argsVar = context.newVariable();
-                VariableExpr argsVarRef1 = new VariableExpr(argsVar);
-                argsVarRef1.setSourceLocation(expr.getSourceLocation());
-                LetClause c = new LetClause(argsVarRef1, argsListExpr);
-                c.setSourceLocation(expr.getSourceLocation());
-                clauses.add(c);
-
-                VariableExpr argsVarRef2 = new VariableExpr(argsVar);
-                argsVarRef2.setSourceLocation(expr.getSourceLocation());
-                subst.addSubstituion(new VariableExpr(paramVarargs), argsVarRef2);
-            } else {
-                if (paramList.size() != argCount) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
-                            paramList.size());
-                }
-                for (int i = 0; i < argCount; i++) {
-                    subst.addSubstituion(new VariableExpr(paramList.get(i)), argVars.get(i));
-                }
-            }
-
-            Pair<ILangExpression, VariableSubstitutionEnvironment> p2 =
-                    implem.getFuncBody().accept(cloneVisitor, subst);
-            Expression resExpr;
-            if (clauses.isEmpty()) {
-                resExpr = (Expression) p2.first;
-            } else {
-                resExpr = generateQueryExpression(clauses, (Expression) p2.first);
-            }
-            return new Pair<>(true, resExpr);
         }
+        if (f.hasAggregateFilterExpr()) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_FILTER_CLAUSE, f.getSourceLocation());
+        }
+        FunctionDecl implem = usedUDFs.get(fs);
+        if (implem == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, f.getSourceLocation(), fs.toString());
+        }
+        // it's one of the functions we want to inline
+        List<Expression> argList = f.getExprList();
+        int argCount = argList.size();
+        List<LetClause> clauses = new ArrayList<>(argCount + 1);
+        List<Expression> argVars = new ArrayList<>(argCount);
+        for (Expression e : f.getExprList()) {
+            // Obs: we could do smth about passing also literals, or let
+            // variable inlining to take care of this.
+            VarIdentifier argVar;
+            if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
+                argVar = ((VariableExpr) e).getVar();
+            } else {
+                SourceLocation sourceLoc = e.getSourceLocation();
+                argVar = context.newVariable();
+                Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
+                        e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
+                VariableExpr newVRef1 = new VariableExpr(argVar);
+                newVRef1.setSourceLocation(sourceLoc);
+                LetClause c = new LetClause(newVRef1, (Expression) p1.first);
+                c.setSourceLocation(sourceLoc);
+                clauses.add(c);
+            }
+
+            VariableExpr argVarExpr = new VariableExpr(argVar);
+            argVarExpr.setSourceLocation(e.getSourceLocation());
+            argVars.add(argVarExpr);
+        }
+
+        VariableSubstitutionEnvironment subst = new VariableSubstitutionEnvironment();
+        List<VarIdentifier> paramList = implem.getParamList();
+        if (implem.getSignature().getArity() == FunctionIdentifier.VARARGS) {
+            if (paramList.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
+                        paramList.size());
+            }
+            VarIdentifier paramVarargs = paramList.get(0);
+            CallExpr argsListExpr =
+                    new CallExpr(new FunctionSignature(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR), argVars);
+            argsListExpr.setSourceLocation(expr.getSourceLocation());
+
+            VarIdentifier argsVar = context.newVariable();
+            VariableExpr argsVarRef1 = new VariableExpr(argsVar);
+            argsVarRef1.setSourceLocation(expr.getSourceLocation());
+            LetClause c = new LetClause(argsVarRef1, argsListExpr);
+            c.setSourceLocation(expr.getSourceLocation());
+            clauses.add(c);
+
+            VariableExpr argsVarRef2 = new VariableExpr(argsVar);
+            argsVarRef2.setSourceLocation(expr.getSourceLocation());
+            subst.addSubstituion(new VariableExpr(paramVarargs), argsVarRef2);
+        } else {
+            if (paramList.size() != argCount) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
+                        paramList.size());
+            }
+            for (int i = 0; i < argCount; i++) {
+                subst.addSubstituion(new VariableExpr(paramList.get(i)), argVars.get(i));
+            }
+        }
+
+        Expression funcBodyNorm = implem.getNormalizedFuncBody();
+        if (funcBodyNorm == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(), fs.toString());
+        }
+        Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = funcBodyNorm.accept(cloneVisitor, subst);
+        Expression resExpr;
+        if (clauses.isEmpty()) {
+            resExpr = (Expression) p2.first;
+        } else {
+            resExpr = generateQueryExpression(clauses, (Expression) p2.first);
+        }
+        return new Pair<>(true, resExpr);
     }
 
-    protected Pair<Boolean, List<Expression>> inlineUdfsInExprList(List<Expression> exprList, List<FunctionDecl> fds)
+    protected Pair<Boolean, List<Expression>> inlineUdfsInExprList(List<Expression> exprList)
             throws CompilationException {
         List<Expression> newList = new ArrayList<>(exprList.size());
         boolean changed = false;
         for (Expression e : exprList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(e, fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(e);
             newList.add(be.second);
             changed |= be.first;
         }
@@ -391,11 +381,11 @@
     }
 
     private Pair<Boolean, List<GbyVariableExpressionPair>> inlineUdfsInGbyPairList(
-            List<GbyVariableExpressionPair> gbyPairList, List<FunctionDecl> fds) throws CompilationException {
+            List<GbyVariableExpressionPair> gbyPairList) throws CompilationException {
         List<GbyVariableExpressionPair> newList = new ArrayList<>(gbyPairList.size());
         boolean changed = false;
         for (GbyVariableExpressionPair p : gbyPairList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.getExpr(), fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.getExpr());
             newList.add(new GbyVariableExpressionPair(p.getVar(), be.second));
             changed |= be.first;
         }
@@ -403,69 +393,26 @@
     }
 
     protected Pair<Boolean, List<Pair<Expression, Identifier>>> inlineUdfsInFieldList(
-            List<Pair<Expression, Identifier>> fieldList, List<FunctionDecl> fds) throws CompilationException {
+            List<Pair<Expression, Identifier>> fieldList) throws CompilationException {
         List<Pair<Expression, Identifier>> newList = new ArrayList<>(fieldList.size());
         boolean changed = false;
         for (Pair<Expression, Identifier> p : fieldList) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.first, fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(p.first);
             newList.add(new Pair<>(be.second, p.second));
             changed |= be.first;
         }
         return new Pair<>(changed, newList);
     }
 
-    private Pair<Boolean, Map<Expression, VariableExpr>> inlineUdfsInVarMap(Map<Expression, VariableExpr> varMap,
-            List<FunctionDecl> fds) throws CompilationException {
+    private Pair<Boolean, Map<Expression, VariableExpr>> inlineUdfsInVarMap(Map<Expression, VariableExpr> varMap)
+            throws CompilationException {
         Map<Expression, VariableExpr> newMap = new HashMap<>();
         boolean changed = false;
         for (Map.Entry<Expression, VariableExpr> me : varMap.entrySet()) {
-            Pair<Boolean, Expression> be = inlineUdfsInExpr(me.getKey(), fds);
+            Pair<Boolean, Expression> be = inlineUdfsInExpr(me.getKey());
             newMap.put(be.second, me.getValue());
             changed |= be.first;
         }
         return new Pair<>(changed, newMap);
     }
-
-    private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
-        SourceLocation sourceLoc = fnDecl.getSourceLocation();
-
-        DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
-        Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
-        Dataverse fnDataverse;
-        if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
-            fnDataverse = defaultDataverse;
-        } else {
-            try {
-                fnDataverse = metadataProvider.findDataverse(fnDataverseName);
-            } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnDataverseName);
-            }
-        }
-
-        metadataProvider.setDefaultDataverse(fnDataverse);
-        try {
-            Query wrappedQuery = new Query(false);
-            wrappedQuery.setSourceLocation(sourceLoc);
-            wrappedQuery.setBody(fnDecl.getFuncBody());
-            wrappedQuery.setTopLevel(false);
-            IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
-            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context, true,
-                    fnDecl.getParamList());
-            return wrappedQuery.getBody();
-        } catch (CompilationException e) {
-            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, fnDecl.getSignature(),
-                    e.getMessage());
-        } finally {
-            metadataProvider.setDefaultDataverse(defaultDataverse);
-        }
-    }
-
-    private static FunctionDecl findFuncDeclaration(FunctionSignature fid, List<FunctionDecl> sequence) {
-        for (FunctionDecl f : sequence) {
-            if (f.getSignature().equals(fid)) {
-                return f;
-            }
-        }
-        return null;
-    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index 1fcf822..4b30c97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -178,7 +178,7 @@
         }
 
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = fd.getFuncBody().accept(this, env);
-        FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first);
+        FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first, fd.isStored());
         newF.setSourceLocation(fd.getSourceLocation());
         return new Pair<>(newF, env);
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index 3ad0f1b..25f4103 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -19,10 +19,9 @@
 
 package org.apache.asterix.lang.common.visitor;
 
-import java.util.LinkedHashSet;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.Expression;
@@ -56,9 +55,13 @@
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class GatherFunctionCallsVisitor extends AbstractQueryExpressionVisitor<Void, Void> {
+public abstract class GatherFunctionCallsVisitor extends AbstractQueryExpressionVisitor<Void, Void> {
 
-    protected final Set<AbstractCallExpression> calls = new LinkedHashSet<>();
+    protected final Collection<? super AbstractCallExpression> calls;
+
+    protected GatherFunctionCallsVisitor(Collection<? super AbstractCallExpression> calls) {
+        this.calls = calls;
+    }
 
     @Override
     public Void visit(CallExpr callExpr, Void arg) throws CompilationException {
@@ -240,10 +243,6 @@
         return null;
     }
 
-    public Set<AbstractCallExpression> getCalls() {
-        return calls;
-    }
-
     @Override
     public Void visit(FunctionDecl fd, Void arg) throws CompilationException {
         return null;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 356a683..8dd206f 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -19,15 +19,13 @@
 package org.apache.asterix.lang.sqlpp.rewrites;
 
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
 
@@ -36,11 +34,16 @@
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars) throws CompilationException {
+    public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
+        if (inlineUdfs) {
+            // When rewriting function body we do not inline UDFs into it.
+            // The main query rewriter will inline everything later, when it processes the query
+            throw new CompilationException(ErrorCode.ILLEGAL_STATE, topStatement.getSourceLocation(), "inlineUdfs");
+        }
+
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
+        setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
 
         // Resolves function calls
         resolveFunctionCalls();
@@ -93,8 +96,5 @@
 
         // Rewrites RIGHT OUTER JOINs into LEFT OUTER JOINs if possible
         rewriteRightJoins();
-
-        // Inlines functions recursively.
-        inlineDeclaredUdfs(inlineUdfs);
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index af1bb41..738fc69 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -18,48 +18,31 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites;
 
-import java.util.ArrayList;
+import java.util.ArrayDeque;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashSet;
-import java.util.List;
+import java.util.LinkedHashMap;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IReturningStatement;
-import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.AbstractCallExpression;
-import org.apache.asterix.lang.common.expression.ListSliceExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
-import org.apache.asterix.lang.common.parser.FunctionParser;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
-import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.FromTerm;
-import org.apache.asterix.lang.sqlpp.clause.HavingClause;
-import org.apache.asterix.lang.sqlpp.clause.JoinClause;
-import org.apache.asterix.lang.sqlpp.clause.NestClause;
-import org.apache.asterix.lang.sqlpp.clause.Projection;
-import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
-import org.apache.asterix.lang.sqlpp.clause.SelectClause;
-import org.apache.asterix.lang.sqlpp.clause.SelectElement;
-import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
-import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
-import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
-import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
-import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
-import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
@@ -68,6 +51,7 @@
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseAggregateExtractionVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseExpressionVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppFunctionCallResolverVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGatherFunctionCallsVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByAggregationSugarVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupingSetsVisitor;
@@ -79,12 +63,12 @@
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppWindowRewriteVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SubstituteGroupbyExpressionWithVariableVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.VariableCheckAndRewriteVisitor;
-import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
-import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.Function;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.util.LogRedactionUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -96,41 +80,38 @@
     public static final String INLINE_WITH_OPTION = "inline_with";
     private static final boolean INLINE_WITH_OPTION_DEFAULT = true;
     private final IParserFactory parserFactory;
-    private final FunctionParser functionParser;
-    private IReturningStatement topExpr;
-    private List<FunctionDecl> declaredFunctions;
+    private SqlppFunctionBodyRewriter functionBodyRewriter;
+    private IReturningStatement topStatement;
     private LangRewritingContext context;
     private MetadataProvider metadataProvider;
     private Collection<VarIdentifier> externalVars;
+    private boolean allowNonStoredUdfCalls;
+    private boolean inlineUdfs;
     private boolean isLogEnabled;
 
     public SqlppQueryRewriter(IParserFactory parserFactory) {
         this.parserFactory = parserFactory;
-        functionParser = new FunctionParser(parserFactory);
     }
 
-    protected void setup(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context, Collection<VarIdentifier> externalVars)
+    protected void setup(LangRewritingContext context, IReturningStatement topStatement,
+            Collection<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls, boolean inlineUdfs)
             throws CompilationException {
-        this.topExpr = topExpr;
         this.context = context;
-        this.declaredFunctions = declaredFunctions;
-        this.metadataProvider = metadataProvider;
+        this.metadataProvider = context.getMetadataProvider();
+        this.topStatement = topStatement;
         this.externalVars = externalVars != null ? externalVars : Collections.emptyList();
+        this.allowNonStoredUdfCalls = allowNonStoredUdfCalls;
+        this.inlineUdfs = inlineUdfs;
         this.isLogEnabled = LOGGER.isTraceEnabled();
         logExpression("Starting AST rewrites on", "");
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
-            Collection<VarIdentifier> externalVars) throws CompilationException {
-        if (topStatement == null) {
-            return;
-        }
+    public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
+            boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
 
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
+        setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
 
         // Resolves function calls
         resolveFunctionCalls();
@@ -182,7 +163,7 @@
         rewriteRightJoins();
 
         // Inlines functions.
-        inlineDeclaredUdfs(inlineUdfs);
+        loadAndInlineDeclaredUdfs();
 
         // Rewrites SQL++ core aggregate function names into internal names
         rewriteSpecialFunctionNames();
@@ -214,7 +195,7 @@
 
     protected void resolveFunctionCalls() throws CompilationException {
         SqlppFunctionCallResolverVisitor visitor =
-                new SqlppFunctionCallResolverVisitor(metadataProvider, declaredFunctions);
+                new SqlppFunctionCallResolverVisitor(context, allowNonStoredUdfCalls);
         rewriteTopExpr(visitor, null);
     }
 
@@ -308,53 +289,40 @@
         rewriteTopExpr(visitor, null);
     }
 
-    protected void inlineDeclaredUdfs(boolean inlineUdfs) throws CompilationException {
-        List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
-        for (FunctionDecl fdecl : declaredFunctions) {
-            funIds.add(fdecl.getSignature());
-        }
-
-        List<FunctionDecl> usedStoredFunctionDecls = new ArrayList<>();
-        for (Expression topLevelExpr : topExpr.getDirectlyEnclosedExpressions()) {
-            usedStoredFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr,
-                    funIds, null, this::getFunctionCalls, functionParser, context.getWarningCollector()));
-        }
-        declaredFunctions.addAll(usedStoredFunctionDecls);
-        if (inlineUdfs && !declaredFunctions.isEmpty()) {
-            SqlppFunctionBodyRewriterFactory functionBodyRewriterFactory =
-                    new SqlppFunctionBodyRewriterFactory(parserFactory);
-            SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, functionBodyRewriterFactory,
-                    declaredFunctions, metadataProvider);
-            while (rewriteTopExpr(visitor, declaredFunctions)) {
+    protected void loadAndInlineDeclaredUdfs() throws CompilationException {
+        Map<FunctionSignature, FunctionDecl> udfs = fetchUserDefinedFunctions(topStatement);
+        FunctionUtil.checkFunctionRecursion(udfs, SqlppGatherFunctionCallsVisitor::new,
+                topStatement.getSourceLocation());
+        if (!udfs.isEmpty() && inlineUdfs) {
+            SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, udfs);
+            while (rewriteTopExpr(visitor, null)) {
                 // loop until no more changes
             }
         }
-        declaredFunctions.removeAll(usedStoredFunctionDecls);
     }
 
     private <R, T> R rewriteTopExpr(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
-        R result = topExpr.accept(visitor, arg);
+        R result = topStatement.accept(visitor, arg);
         logExpression(">>>> AST After", visitor.getClass().getSimpleName());
         return result;
     }
 
     private void logExpression(String p0, String p1) throws CompilationException {
         if (isLogEnabled) {
-            LOGGER.trace("{} {}\n{}", p0, p1, LogRedactionUtil.userData(SqlppAstPrintUtil.toString(topExpr)));
+            LOGGER.trace("{} {}\n{}", p0, p1, LogRedactionUtil.userData(SqlppAstPrintUtil.toString(topStatement)));
         }
     }
 
     @Override
-    public Set<AbstractCallExpression> getFunctionCalls(Expression expression) throws CompilationException {
-        GatherFunctionCalls gfc = new GatherFunctionCalls();
+    public void getFunctionCalls(Expression expression, Collection<? super AbstractCallExpression> outCalls)
+            throws CompilationException {
+        SqlppGatherFunctionCallsVisitor gfc = new SqlppGatherFunctionCallsVisitor(outCalls);
         expression.accept(gfc, null);
-        return gfc.getCalls();
     }
 
     @Override
     public Set<VariableExpr> getExternalVariables(Expression expr) throws CompilationException {
         Set<VariableExpr> freeVars = SqlppVariableUtil.getFreeVariables(expr);
-
         Set<VariableExpr> extVars = new HashSet<>();
         for (VariableExpr ve : freeVars) {
             if (SqlppVariableUtil.isExternalVariableReference(ve)) {
@@ -364,189 +332,104 @@
         return extVars;
     }
 
-    private static class GatherFunctionCalls extends GatherFunctionCallsVisitor implements ISqlppVisitor<Void, Void> {
+    private Map<FunctionSignature, FunctionDecl> fetchUserDefinedFunctions(IReturningStatement topExpr)
+            throws CompilationException {
+        Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
 
-        public GatherFunctionCalls() {
+        Deque<AbstractCallExpression> workQueue = new ArrayDeque<>();
+        SqlppGatherFunctionCallsVisitor gfc = new SqlppGatherFunctionCallsVisitor(workQueue);
+        for (Expression expr : topExpr.getDirectlyEnclosedExpressions()) {
+            expr.accept(gfc, null);
+        }
+        AbstractCallExpression fnCall;
+        while ((fnCall = workQueue.poll()) != null) {
+            switch (fnCall.getKind()) {
+                case CALL_EXPRESSION:
+                    FunctionSignature fs = fnCall.getFunctionSignature();
+                    DataverseName fsDataverse = fs.getDataverseName();
+                    if (fsDataverse == null) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fnCall.getSourceLocation(),
+                                fs);
+                    }
+                    if (FunctionUtil.isBuiltinFunctionSignature(fs) || udfs.containsKey(fs)) {
+                        continue;
+                    }
+                    FunctionDecl fd = context.getDeclaredFunctions().get(fs);
+                    if (fd == null) {
+                        Function function;
+                        try {
+                            function = metadataProvider.lookupUserDefinedFunction(fs);
+                        } catch (AlgebricksException e) {
+                            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
+                                    fs.toString());
+                        }
+                        if (function == null) {
+                            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
+                                    fs.toString());
+                        }
+                        fd = FunctionUtil.parseStoredFunction(function, parserFactory, context.getWarningCollector(),
+                                fnCall.getSourceLocation());
+                    }
+                    prepareFunction(fd);
+                    udfs.put(fs, fd);
+                    fd.getNormalizedFuncBody().accept(gfc, null);
+                    break;
+                case WINDOW_EXPRESSION:
+                    // there cannot be used-defined window functions
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fnCall.getSourceLocation(),
+                            fnCall.getFunctionSignature().toString(false));
+            }
+        }
+        return udfs;
+    }
+
+    private void prepareFunction(FunctionDecl fd) throws CompilationException {
+        Expression fnNormBody = fd.getNormalizedFuncBody();
+        if (fnNormBody == null) {
+            fnNormBody = rewriteFunctionBody(fd);
+            fd.setNormalizedFuncBody(fnNormBody);
+        }
+    }
+
+    private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
+        DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
+        Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
+        Dataverse fnDataverse;
+        if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
+            fnDataverse = defaultDataverse;
+        } else {
+            try {
+                fnDataverse = metadataProvider.findDataverse(fnDataverseName);
+            } catch (AlgebricksException e) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, fnDecl.getSourceLocation(),
+                        fnDataverseName);
+            }
         }
 
-        @Override
-        public Void visit(FromClause fromClause, Void arg) throws CompilationException {
-            for (FromTerm fromTerm : fromClause.getFromTerms()) {
-                fromTerm.accept(this, arg);
-            }
-            return null;
+        metadataProvider.setDefaultDataverse(fnDataverse);
+        try {
+            Query wrappedQuery = new Query(false);
+            wrappedQuery.setSourceLocation(fnDecl.getSourceLocation());
+            wrappedQuery.setBody(fnDecl.getFuncBody());
+            wrappedQuery.setTopLevel(false);
+            boolean allowNonStoredUdfCalls = !fnDecl.isStored();
+            getFunctionBodyRewriter().rewrite(context, wrappedQuery, allowNonStoredUdfCalls, false,
+                    fnDecl.getParamList());
+            return wrappedQuery.getBody();
+        } catch (CompilationException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, fnDecl.getSignature(),
+                    e.getMessage());
+        } finally {
+            metadataProvider.setDefaultDataverse(defaultDataverse);
         }
+    }
 
-        @Override
-        public Void visit(FromTerm fromTerm, Void arg) throws CompilationException {
-            fromTerm.getLeftExpression().accept(this, arg);
-            for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-                correlateClause.accept(this, arg);
-            }
-            return null;
+    protected SqlppFunctionBodyRewriter getFunctionBodyRewriter() {
+        if (functionBodyRewriter == null) {
+            functionBodyRewriter = new SqlppFunctionBodyRewriter(parserFactory);
         }
-
-        @Override
-        public Void visit(JoinClause joinClause, Void arg) throws CompilationException {
-            joinClause.getRightExpression().accept(this, arg);
-            joinClause.getConditionExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(NestClause nestClause, Void arg) throws CompilationException {
-            nestClause.getRightExpression().accept(this, arg);
-            nestClause.getConditionExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(Projection projection, Void arg) throws CompilationException {
-            if (!projection.star()) {
-                projection.getExpression().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectBlock selectBlock, Void arg) throws CompilationException {
-            if (selectBlock.hasFromClause()) {
-                selectBlock.getFromClause().accept(this, arg);
-            }
-            if (selectBlock.hasLetWhereClauses()) {
-                for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
-                    letWhereClause.accept(this, arg);
-                }
-            }
-            if (selectBlock.hasGroupbyClause()) {
-                selectBlock.getGroupbyClause().accept(this, arg);
-            }
-            if (selectBlock.hasLetHavingClausesAfterGroupby()) {
-                for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
-                    letHavingClause.accept(this, arg);
-                }
-            }
-            selectBlock.getSelectClause().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectClause selectClause, Void arg) throws CompilationException {
-            if (selectClause.selectElement()) {
-                selectClause.getSelectElement().accept(this, arg);
-            } else {
-                selectClause.getSelectRegular().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectElement selectElement, Void arg) throws CompilationException {
-            selectElement.getExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectRegular selectRegular, Void arg) throws CompilationException {
-            for (Projection projection : selectRegular.getProjections()) {
-                projection.accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
-            selectSetOperation.getLeftInput().accept(this, arg);
-            for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
-                setOperationRight.getSetOperationRightInput().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(SelectExpression selectStatement, Void arg) throws CompilationException {
-            if (selectStatement.hasLetClauses()) {
-                for (LetClause letClause : selectStatement.getLetList()) {
-                    letClause.accept(this, arg);
-                }
-            }
-            selectStatement.getSelectSetOperation().accept(this, arg);
-            if (selectStatement.hasOrderby()) {
-                selectStatement.getOrderbyClause().accept(this, arg);
-            }
-            if (selectStatement.hasLimit()) {
-                selectStatement.getLimitClause().accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(UnnestClause unnestClause, Void arg) throws CompilationException {
-            unnestClause.getRightExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(HavingClause havingClause, Void arg) throws CompilationException {
-            havingClause.getFilterExpression().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(CaseExpression caseExpression, Void arg) throws CompilationException {
-            caseExpression.getConditionExpr().accept(this, arg);
-            for (Expression expr : caseExpression.getWhenExprs()) {
-                expr.accept(this, arg);
-            }
-            for (Expression expr : caseExpression.getThenExprs()) {
-                expr.accept(this, arg);
-            }
-            caseExpression.getElseExpr().accept(this, arg);
-            return null;
-        }
-
-        @Override
-        public Void visit(WindowExpression winExpr, Void arg) throws CompilationException {
-            calls.add(winExpr);
-            if (winExpr.hasPartitionList()) {
-                for (Expression expr : winExpr.getPartitionList()) {
-                    expr.accept(this, arg);
-                }
-            }
-            if (winExpr.hasOrderByList()) {
-                for (Expression expr : winExpr.getOrderbyList()) {
-                    expr.accept(this, arg);
-                }
-            }
-            if (winExpr.hasFrameStartExpr()) {
-                winExpr.getFrameStartExpr().accept(this, arg);
-            }
-            if (winExpr.hasFrameEndExpr()) {
-                winExpr.getFrameEndExpr().accept(this, arg);
-            }
-            if (winExpr.hasWindowFieldList()) {
-                for (Pair<Expression, Identifier> p : winExpr.getWindowFieldList()) {
-                    p.first.accept(this, arg);
-                }
-            }
-            if (winExpr.hasAggregateFilterExpr()) {
-                winExpr.getAggregateFilterExpr().accept(this, arg);
-            }
-            for (Expression expr : winExpr.getExprList()) {
-                expr.accept(this, arg);
-            }
-            return null;
-        }
-
-        @Override
-        public Void visit(ListSliceExpression expression, Void arg) throws CompilationException {
-            expression.getExpr().accept(this, arg);
-            expression.getStartIndexExpression().accept(this, arg);
-
-            if (expression.hasEndExpression()) {
-                expression.getEndIndexExpression().accept(this, arg);
-            }
-            return null;
-        }
+        return functionBodyRewriter;
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
index 7666032..38b66e2 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
-import java.util.List;
-import java.util.Set;
 import java.util.function.BiFunction;
 
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -28,35 +26,35 @@
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.expression.CallExpr;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 
 public final class SqlppFunctionCallResolverVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
-    private final MetadataProvider metadataProvider;
+    private final LangRewritingContext context;
 
-    private final Set<FunctionSignature> declaredFunctions;
+    private final boolean allowNonStoredUdfCalls;
 
     private final BiFunction<String, Integer, FunctionSignature> builtinFunctionResolver;
 
     private final BiFunction<String, Integer, FunctionSignature> callExprResolver;
 
-    public SqlppFunctionCallResolverVisitor(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions) {
-        this.metadataProvider = metadataProvider;
-        this.declaredFunctions = FunctionUtil.getFunctionSignatures(declaredFunctions);
-        this.builtinFunctionResolver = FunctionUtil.createBuiltinFunctionResolver(metadataProvider);
+    public SqlppFunctionCallResolverVisitor(LangRewritingContext context, boolean allowNonStoredUdfCalls) {
+        this.context = context;
+        this.allowNonStoredUdfCalls = allowNonStoredUdfCalls;
+        this.builtinFunctionResolver = FunctionUtil.createBuiltinFunctionResolver(context.getMetadataProvider());
         this.callExprResolver = this::resolveCallExpr;
     }
 
     @Override
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         FunctionSignature fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(),
-                callExpr.getSourceLocation(), metadataProvider, declaredFunctions, callExprResolver);
+                callExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, true,
+                context.getDeclaredFunctions(), allowNonStoredUdfCalls);
         callExpr.setFunctionSignature(fs);
         return super.visit(callExpr, arg);
     }
@@ -64,7 +62,7 @@
     @Override
     public Expression visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
         FunctionSignature fs = FunctionUtil.resolveFunctionCall(winExpr.getFunctionSignature(),
-                winExpr.getSourceLocation(), metadataProvider, declaredFunctions, callExprResolver);
+                winExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, false, null, false);
         winExpr.setFunctionSignature(fs);
         return super.visit(winExpr, arg);
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
new file mode 100644
index 0000000..8559c96
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
@@ -0,0 +1,238 @@
+/*
+ * 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.lang.sqlpp.rewrites.visitor;
+
+import java.util.Collection;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.expression.AbstractCallExpression;
+import org.apache.asterix.lang.common.expression.ListSliceExpression;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
+import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.HavingClause;
+import org.apache.asterix.lang.sqlpp.clause.JoinClause;
+import org.apache.asterix.lang.sqlpp.clause.NestClause;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
+import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
+import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public final class SqlppGatherFunctionCallsVisitor extends GatherFunctionCallsVisitor
+        implements ISqlppVisitor<Void, Void> {
+
+    public SqlppGatherFunctionCallsVisitor(Collection<? super AbstractCallExpression> calls) {
+        super(calls);
+    }
+
+    @Override
+    public Void visit(FromClause fromClause, Void arg) throws CompilationException {
+        for (FromTerm fromTerm : fromClause.getFromTerms()) {
+            fromTerm.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(FromTerm fromTerm, Void arg) throws CompilationException {
+        fromTerm.getLeftExpression().accept(this, arg);
+        for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
+            correlateClause.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(JoinClause joinClause, Void arg) throws CompilationException {
+        joinClause.getRightExpression().accept(this, arg);
+        joinClause.getConditionExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(NestClause nestClause, Void arg) throws CompilationException {
+        nestClause.getRightExpression().accept(this, arg);
+        nestClause.getConditionExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(Projection projection, Void arg) throws CompilationException {
+        if (!projection.star()) {
+            projection.getExpression().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectBlock selectBlock, Void arg) throws CompilationException {
+        if (selectBlock.hasFromClause()) {
+            selectBlock.getFromClause().accept(this, arg);
+        }
+        if (selectBlock.hasLetWhereClauses()) {
+            for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
+                letWhereClause.accept(this, arg);
+            }
+        }
+        if (selectBlock.hasGroupbyClause()) {
+            selectBlock.getGroupbyClause().accept(this, arg);
+        }
+        if (selectBlock.hasLetHavingClausesAfterGroupby()) {
+            for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
+                letHavingClause.accept(this, arg);
+            }
+        }
+        selectBlock.getSelectClause().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectClause selectClause, Void arg) throws CompilationException {
+        if (selectClause.selectElement()) {
+            selectClause.getSelectElement().accept(this, arg);
+        } else {
+            selectClause.getSelectRegular().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectElement selectElement, Void arg) throws CompilationException {
+        selectElement.getExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectRegular selectRegular, Void arg) throws CompilationException {
+        for (Projection projection : selectRegular.getProjections()) {
+            projection.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
+        selectSetOperation.getLeftInput().accept(this, arg);
+        for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
+            setOperationRight.getSetOperationRightInput().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(SelectExpression selectStatement, Void arg) throws CompilationException {
+        if (selectStatement.hasLetClauses()) {
+            for (LetClause letClause : selectStatement.getLetList()) {
+                letClause.accept(this, arg);
+            }
+        }
+        selectStatement.getSelectSetOperation().accept(this, arg);
+        if (selectStatement.hasOrderby()) {
+            selectStatement.getOrderbyClause().accept(this, arg);
+        }
+        if (selectStatement.hasLimit()) {
+            selectStatement.getLimitClause().accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(UnnestClause unnestClause, Void arg) throws CompilationException {
+        unnestClause.getRightExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(HavingClause havingClause, Void arg) throws CompilationException {
+        havingClause.getFilterExpression().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(CaseExpression caseExpression, Void arg) throws CompilationException {
+        caseExpression.getConditionExpr().accept(this, arg);
+        for (Expression expr : caseExpression.getWhenExprs()) {
+            expr.accept(this, arg);
+        }
+        for (Expression expr : caseExpression.getThenExprs()) {
+            expr.accept(this, arg);
+        }
+        caseExpression.getElseExpr().accept(this, arg);
+        return null;
+    }
+
+    @Override
+    public Void visit(WindowExpression winExpr, Void arg) throws CompilationException {
+        calls.add(winExpr);
+        if (winExpr.hasPartitionList()) {
+            for (Expression expr : winExpr.getPartitionList()) {
+                expr.accept(this, arg);
+            }
+        }
+        if (winExpr.hasOrderByList()) {
+            for (Expression expr : winExpr.getOrderbyList()) {
+                expr.accept(this, arg);
+            }
+        }
+        if (winExpr.hasFrameStartExpr()) {
+            winExpr.getFrameStartExpr().accept(this, arg);
+        }
+        if (winExpr.hasFrameEndExpr()) {
+            winExpr.getFrameEndExpr().accept(this, arg);
+        }
+        if (winExpr.hasWindowFieldList()) {
+            for (Pair<Expression, Identifier> p : winExpr.getWindowFieldList()) {
+                p.first.accept(this, arg);
+            }
+        }
+        if (winExpr.hasAggregateFilterExpr()) {
+            winExpr.getAggregateFilterExpr().accept(this, arg);
+        }
+        for (Expression expr : winExpr.getExprList()) {
+            expr.accept(this, arg);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(ListSliceExpression expression, Void arg) throws CompilationException {
+        expression.getExpr().accept(this, arg);
+        expression.getStartIndexExpression().accept(this, arg);
+
+        if (expression.hasEndExpression()) {
+            expression.getEndIndexExpression().accept(this, arg);
+        }
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
index 4fb17f0..5200b94 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
@@ -23,9 +23,9 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
-import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.ListSliceExpression;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -52,26 +52,18 @@
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppCloneAndSubstituteVariablesVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor
-        implements ISqlppVisitor<Boolean, List<FunctionDecl>> {
+public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor implements ISqlppVisitor<Boolean, Void> {
 
     /**
      * @param context,
      *            manages ids of variables and guarantees uniqueness of variables.
-     * @param rewriterFactory,
-     *            a rewrite factory for rewriting user-defined functions.
-     * @param declaredFunctions,
-     *            a list of declared functions associated with the query.
-     * @param metadataProvider,
-     *            providing the definition of created (i.e., stored) user-defined functions.
+     * @param usedUDFs,
+     *            user defined functions used by this query.
      */
-    public SqlppInlineUdfsVisitor(LangRewritingContext context, IRewriterFactory rewriterFactory,
-            List<FunctionDecl> declaredFunctions, MetadataProvider metadataProvider) {
-        super(context, rewriterFactory, declaredFunctions, metadataProvider,
-                new SqlppCloneAndSubstituteVariablesVisitor(context));
+    public SqlppInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs) {
+        super(context, usedUDFs, new SqlppCloneAndSubstituteVariablesVisitor(context));
     }
 
     @Override
@@ -82,220 +74,218 @@
     }
 
     @Override
-    public Boolean visit(FromClause fromClause, List<FunctionDecl> func) throws CompilationException {
+    public Boolean visit(FromClause fromClause, Void arg) throws CompilationException {
         boolean changed = false;
         for (FromTerm fromTerm : fromClause.getFromTerms()) {
-            changed |= fromTerm.accept(this, func);
+            changed |= fromTerm.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(FromTerm fromTerm, List<FunctionDecl> func) throws CompilationException {
+    public Boolean visit(FromTerm fromTerm, Void arg) throws CompilationException {
         boolean changed = false;
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(fromTerm.getLeftExpression(), func);
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(fromTerm.getLeftExpression());
         fromTerm.setLeftExpression(p.second);
         changed |= p.first;
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-            changed |= correlateClause.accept(this, func);
+            changed |= correlateClause.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(JoinClause joinClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(joinClause.getRightExpression(), funcs);
+    public Boolean visit(JoinClause joinClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(joinClause.getRightExpression());
         joinClause.setRightExpression(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(joinClause.getConditionExpression(), funcs);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(joinClause.getConditionExpression());
         joinClause.setConditionExpression(p2.second);
         return p1.first || p2.first;
     }
 
     @Override
-    public Boolean visit(NestClause nestClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(nestClause.getRightExpression(), funcs);
+    public Boolean visit(NestClause nestClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p1 = inlineUdfsInExpr(nestClause.getRightExpression());
         nestClause.setRightExpression(p1.second);
-        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(nestClause.getConditionExpression(), funcs);
+        Pair<Boolean, Expression> p2 = inlineUdfsInExpr(nestClause.getConditionExpression());
         nestClause.setConditionExpression(p2.second);
         return p1.first || p2.first;
     }
 
     @Override
-    public Boolean visit(Projection projection, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(Projection projection, Void arg) throws CompilationException {
         if (projection.star()) {
             return false;
         }
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(projection.getExpression(), funcs);
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(projection.getExpression());
         projection.setExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(SelectBlock selectBlock, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectBlock selectBlock, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectBlock.hasFromClause()) {
-            changed |= selectBlock.getFromClause().accept(this, funcs);
+            changed |= selectBlock.getFromClause().accept(this, arg);
         }
         if (selectBlock.hasLetWhereClauses()) {
             for (AbstractClause letWhereClause : selectBlock.getLetWhereList()) {
-                changed |= letWhereClause.accept(this, funcs);
+                changed |= letWhereClause.accept(this, arg);
             }
         }
         if (selectBlock.hasGroupbyClause()) {
-            changed |= selectBlock.getGroupbyClause().accept(this, funcs);
+            changed |= selectBlock.getGroupbyClause().accept(this, arg);
         }
         if (selectBlock.hasLetHavingClausesAfterGroupby()) {
             for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
-                changed |= letHavingClause.accept(this, funcs);
+                changed |= letHavingClause.accept(this, arg);
             }
         }
-        changed |= selectBlock.getSelectClause().accept(this, funcs);
+        changed |= selectBlock.getSelectClause().accept(this, arg);
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectClause selectClause, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectClause selectClause, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectClause.selectElement()) {
-            changed |= selectClause.getSelectElement().accept(this, funcs);
+            changed |= selectClause.getSelectElement().accept(this, arg);
         } else {
-            changed |= selectClause.getSelectRegular().accept(this, funcs);
+            changed |= selectClause.getSelectRegular().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectElement selectElement, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(selectElement.getExpression(), funcs);
+    public Boolean visit(SelectElement selectElement, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(selectElement.getExpression());
         selectElement.setExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(SelectRegular selectRegular, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectRegular selectRegular, Void arg) throws CompilationException {
         boolean changed = false;
         for (Projection projection : selectRegular.getProjections()) {
-            changed |= projection.accept(this, funcs);
+            changed |= projection.accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectSetOperation selectSetOperation, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectSetOperation selectSetOperation, Void arg) throws CompilationException {
         boolean changed = false;
-        changed |= selectSetOperation.getLeftInput().accept(this, funcs);
+        changed |= selectSetOperation.getLeftInput().accept(this, arg);
         for (SetOperationRight right : selectSetOperation.getRightInputs()) {
-            changed |= right.getSetOperationRightInput().accept(this, funcs);
+            changed |= right.getSetOperationRightInput().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(SelectExpression selectExpression, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(SelectExpression selectExpression, Void arg) throws CompilationException {
         boolean changed = false;
         if (selectExpression.hasLetClauses()) {
             for (LetClause letClause : selectExpression.getLetList()) {
-                changed |= letClause.accept(this, funcs);
+                changed |= letClause.accept(this, arg);
             }
         }
-        changed |= selectExpression.getSelectSetOperation().accept(this, funcs);
+        changed |= selectExpression.getSelectSetOperation().accept(this, arg);
         if (selectExpression.hasOrderby()) {
-            changed |= selectExpression.getOrderbyClause().accept(this, funcs);
+            changed |= selectExpression.getOrderbyClause().accept(this, arg);
         }
         if (selectExpression.hasLimit()) {
-            changed |= selectExpression.getLimitClause().accept(this, funcs);
+            changed |= selectExpression.getLimitClause().accept(this, arg);
         }
         return changed;
     }
 
     @Override
-    public Boolean visit(UnnestClause unnestClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(unnestClause.getRightExpression(), funcs);
+    public Boolean visit(UnnestClause unnestClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(unnestClause.getRightExpression());
         unnestClause.setRightExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(HavingClause havingClause, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> p = inlineUdfsInExpr(havingClause.getFilterExpression(), funcs);
+    public Boolean visit(HavingClause havingClause, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> p = inlineUdfsInExpr(havingClause.getFilterExpression());
         havingClause.setFilterExpression(p.second);
         return p.first;
     }
 
     @Override
-    public Boolean visit(CaseExpression caseExpr, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> result = inlineUdfsInExpr(caseExpr.getConditionExpr(), funcs);
+    public Boolean visit(CaseExpression caseExpr, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> result = inlineUdfsInExpr(caseExpr.getConditionExpr());
         caseExpr.setConditionExpr(result.second);
         boolean inlined = result.first;
 
-        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(caseExpr.getWhenExprs(), funcs);
+        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(caseExpr.getWhenExprs());
         inlined = inlined || inlinedList.first;
         caseExpr.setWhenExprs(inlinedList.second);
 
-        inlinedList = inlineUdfsInExprList(caseExpr.getThenExprs(), funcs);
+        inlinedList = inlineUdfsInExprList(caseExpr.getThenExprs());
         inlined = inlined || inlinedList.first;
         caseExpr.setThenExprs(inlinedList.second);
 
-        result = inlineUdfsInExpr(caseExpr.getElseExpr(), funcs);
+        result = inlineUdfsInExpr(caseExpr.getElseExpr());
         caseExpr.setElseExpr(result.second);
         return inlined || result.first;
     }
 
     @Override
-    public Boolean visit(WindowExpression winExpr, List<FunctionDecl> funcs) throws CompilationException {
+    public Boolean visit(WindowExpression winExpr, Void arg) throws CompilationException {
         boolean inlined = false;
         if (winExpr.hasPartitionList()) {
-            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getPartitionList(), funcs);
+            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getPartitionList());
             winExpr.setPartitionList(inlinedList.second);
             inlined = inlinedList.first;
         }
         if (winExpr.hasOrderByList()) {
-            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getOrderbyList(), funcs);
+            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getOrderbyList());
             winExpr.setOrderbyList(inlinedList.second);
             inlined |= inlinedList.first;
         }
         if (winExpr.hasFrameStartExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameStartExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameStartExpr());
             winExpr.setFrameStartExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
         if (winExpr.hasFrameEndExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameEndExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameEndExpr());
             winExpr.setFrameEndExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
         if (winExpr.hasWindowFieldList()) {
             Pair<Boolean, List<Pair<Expression, Identifier>>> inlinedList =
-                    inlineUdfsInFieldList(winExpr.getWindowFieldList(), funcs);
+                    inlineUdfsInFieldList(winExpr.getWindowFieldList());
             winExpr.setWindowFieldList(inlinedList.second);
             inlined |= inlinedList.first;
         }
         if (winExpr.hasAggregateFilterExpr()) {
-            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getAggregateFilterExpr(), funcs);
+            Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getAggregateFilterExpr());
             winExpr.setAggregateFilterExpr(inlinedExpr.second);
             inlined |= inlinedExpr.first;
         }
-        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getExprList(), funcs);
+        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getExprList());
         winExpr.setExprList(inlinedList.second);
         inlined |= inlinedList.first;
         return inlined;
     }
 
     @Override
-    public Boolean visit(ListSliceExpression expression, List<FunctionDecl> funcs) throws CompilationException {
-        Pair<Boolean, Expression> expressionResult = inlineUdfsInExpr(expression.getExpr(), funcs);
+    public Boolean visit(ListSliceExpression expression, Void arg) throws CompilationException {
+        Pair<Boolean, Expression> expressionResult = inlineUdfsInExpr(expression.getExpr());
         expression.setExpr(expressionResult.second);
         boolean inlined = expressionResult.first;
 
-        Pair<Boolean, Expression> startIndexExpressResult =
-                inlineUdfsInExpr(expression.getStartIndexExpression(), funcs);
+        Pair<Boolean, Expression> startIndexExpressResult = inlineUdfsInExpr(expression.getStartIndexExpression());
         expression.setStartIndexExpression(startIndexExpressResult.second);
         inlined |= startIndexExpressResult.first;
 
         // End index expression can be null (optional)
         if (expression.hasEndExpression()) {
-            Pair<Boolean, Expression> endIndexExpressionResult =
-                    inlineUdfsInExpr(expression.getEndIndexExpression(), funcs);
+            Pair<Boolean, Expression> endIndexExpressionResult = inlineUdfsInExpr(expression.getEndIndexExpression());
             expression.setEndIndexExpression(endIndexExpressionResult.second);
             inlined |= endIndexExpressionResult.first;
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index b5375d2..998d8e6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -257,8 +257,8 @@
 
     @Override
     public FunctionDecl visit(FunctionDecl fd, Void arg) throws CompilationException {
-        FunctionDecl copy =
-                new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        FunctionDecl copy = new FunctionDecl(fd.getSignature(), fd.getParamList(),
+                (Expression) fd.getFuncBody().accept(this, arg), fd.isStored());
         copy.setSourceLocation(fd.getSourceLocation());
         return copy;
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index faee227..b93ab87 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -384,7 +384,7 @@
     }
 
     @Override
-    public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames)
+    public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
       throws CompilationException {
         return parseImpl(new ParseFunction<FunctionDecl>() {
             @Override
@@ -399,7 +399,7 @@
                 Expression functionBodyExpr = SQLPPParser.this.FunctionBody();
                 removeCurrentScope();
                 defaultDataverse = dataverse;
-                return new FunctionDecl(signature, paramVars, functionBodyExpr);
+                return new FunctionDecl(signature, paramVars, functionBodyExpr, isStored);
             }
         });
     }
@@ -2701,7 +2701,7 @@
     for (Pair<VarIdentifier,TypeExpression> p: paramList) {
         params.add(p.getFirst());
     }
-    FunctionDecl stmt = new FunctionDecl(signature, params, funcBody);
+    FunctionDecl stmt = new FunctionDecl(signature, params, funcBody, false);
     removeCurrentScope();
     return addSourceLocation(stmt, startToken);
   }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
Gerrit-Change-Number: 10905
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Hello Jenkins, 

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

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905

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

Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................

[ASTERIXDB-2871][COMP] Improve UDF recursion detection

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

Details:
- Improve UDF recursion detection
- Rewrite UDF body expression only once
- Add testcases

Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
---
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/translator/QueryTranslator.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfInFLOWGR.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/IfThenElse.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR1.ast
M asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.7.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.8.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf37_recursion/udf37_recursion.9.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf39_illegal_call/udf39_illegal_call.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.4.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf38_no_recursion/udf38_no_recursion.6.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/group-by/grouping-sets-2/grouping-sets-2.17.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/group-by/grouping-sets-2/grouping-sets-2.8.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/ifthenelse_01/ifthenelse_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/like_01/like_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/like_null/like_null.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q12_shipping/q12_shipping.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
D asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/FunctionParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
A asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
60 files changed, 1,624 insertions(+), 1,187 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/05/10905/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

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

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905 )

Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................


Patch Set 5: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
Gerrit-Change-Number: 10905
Gerrit-PatchSet: 5
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 08 Apr 2021 20:19:49 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2871][COMP] Improve UDF recursion detection

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905 )

Change subject: [ASTERIXDB-2871][COMP] Improve UDF recursion detection
......................................................................


Patch Set 4: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10905
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: I7886404bfa47d1d89674df1200f30ad6ad0e0fc6
Gerrit-Change-Number: 10905
Gerrit-PatchSet: 4
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 07 Apr 2021 21:40:32 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment