[ASTERIXDB-2180][FUN] Prevent dropping of entities used by functions

Add dependencies to Functional Metadata
Check dependencies before dropping datasets or functions
Add tests

Change-Id: I2f08ff150dfd57432b88381c507814ddb57bd67b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2253
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
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 1257a61..4428e05 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
@@ -180,7 +180,7 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output)
+            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs)
             throws CompilationException {
         if (q == null) {
             return null;
@@ -193,7 +193,7 @@
             printPlanPostfix(output);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()));
+        rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()), inlineUdfs);
         return new Pair<>(q, q.getVarCounter());
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index b013c60..f2933f4 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -248,7 +248,7 @@
                     }
                     Function f = new Function(dataverse, libraryName + "#" + function.getName().trim(), args.size(),
                             args, function.getReturnType().trim(), function.getDefinition().trim(),
-                            library.getLanguage().trim(), function.getFunctionType().trim());
+                            library.getLanguage().trim(), function.getFunctionType().trim(), null);
                     MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
                     if (LOGGER.isInfoEnabled()) {
                         LOGGER.info("Installed function: " + libraryName + "#" + function.getName().trim());
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 70f72a5..9b96883 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
@@ -124,6 +124,7 @@
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.util.MergePolicyUtils;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
@@ -497,8 +498,8 @@
             throw new CompilationException("Unknown compaction policy: " + compactionPolicy);
         }
         String compactionPolicyFactoryClassName = compactionPolicyEntity.getClassName();
-        ILSMMergePolicyFactory mergePolicyFactory = (ILSMMergePolicyFactory) Class
-                .forName(compactionPolicyFactoryClassName).newInstance();
+        ILSMMergePolicyFactory mergePolicyFactory =
+                (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
         if (isExternalDataset && mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
             throw new CompilationException("The correlated-prefix merge policy cannot be used with external dataset.");
         }
@@ -585,10 +586,10 @@
                     }
                     ARecordType metaRecType = (ARecordType) metaItemType;
 
-                    List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getPartitioningExprs();
-                    List<Integer> keySourceIndicators = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getKeySourceIndicators();
+                    List<List<String>> partitioningExprs =
+                            ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs();
+                    List<Integer> keySourceIndicators =
+                            ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getKeySourceIndicators();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
@@ -613,8 +614,8 @@
                     String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
                     Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
 
-                    datasetDetails = new ExternalDatasetDetails(adapter, properties, new Date(),
-                            TransactionState.COMMIT);
+                    datasetDetails =
+                            new ExternalDatasetDetails(adapter, properties, new Date(), TransactionState.COMMIT);
                     break;
                 default:
                     throw new CompilationException("Unknown datatype " + dd.getDatasetType());
@@ -705,8 +706,8 @@
     protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset)
             throws CompilationException {
         StringBuilder builder = null;
-        ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                .getActiveNotificationHandler();
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         IActiveEntityEventsListener[] listeners = activeEventHandler.getEventListeners();
         for (IActiveEntityEventsListener listener : listeners) {
             if (listener.isEntityUsingDataset(dataset) && listener.isActive()) {
@@ -809,8 +810,8 @@
 
             for (Pair<List<String>, IndexedTypeExpression> fieldExpr : stmtCreateIndex.getFieldExprs()) {
                 IAType fieldType = null;
-                ARecordType subType = KeyFieldTypeUtil.chooseSource(keySourceIndicators, keyIndex, aRecordType,
-                        metaRecordType);
+                ARecordType subType =
+                        KeyFieldTypeUtil.chooseSource(keySourceIndicators, keyIndex, aRecordType, metaRecordType);
                 boolean isOpen = subType.isOpen();
                 int i = 0;
                 if (fieldExpr.first.size() > 1 && !isOpen) {
@@ -846,8 +847,8 @@
                     if (stmtCreateIndex.hasMetaField()) {
                         throw new AlgebricksException("Typed open index can only be created on the record part");
                     }
-                    Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx,
-                            fieldExpr.second.getType(), indexName, dataverseName);
+                    Map<TypeSignature, IAType> typeMap =
+                            TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second.getType(), indexName, dataverseName);
                     TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
                     fieldType = typeMap.get(typeSignature);
                     overridesFieldTypes = true;
@@ -1062,8 +1063,8 @@
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                 bActiveTxn = true;
                 try {
-                    JobSpecification jobSpec = ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider,
-                            ds);
+                    JobSpecification jobSpec =
+                            ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     runJob(hcc, jobSpec, jobFlags);
@@ -1219,8 +1220,8 @@
                 }
             }
             // # check whether any function in current dataverse is being used by others
-            List<Function> functionsInDataverse = MetadataManager.INSTANCE.getDataverseFunctions(mdTxnCtx,
-                    dataverseName);
+            List<Function> functionsInDataverse =
+                    MetadataManager.INSTANCE.getDataverseFunctions(mdTxnCtx, dataverseName);
             for (Function function : functionsInDataverse) {
                 if (checkWhetherFunctionIsBeingUsed(mdTxnCtx, function.getDataverseName(), function.getName(),
                         function.getArity(), dataverseName)) {
@@ -1231,8 +1232,8 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
             // # disconnect all feeds from any datasets in the dataverse.
-            ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                    .getActiveNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             IActiveEntityEventsListener[] activeListeners = activeEventHandler.getEventListeners();
             for (IActiveEntityEventsListener listener : activeListeners) {
                 EntityId activeEntityId = listener.getEntityId();
@@ -1260,15 +1261,15 @@
                 String datasetName = dataset.getDatasetName();
                 DatasetType dsType = dataset.getDatasetType();
                 if (dsType == DatasetType.INTERNAL) {
-                    List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
+                    List<Index> indexes =
+                            MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (Index index : indexes) {
                         jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset));
                     }
                 } else {
                     // External dataset
-                    List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
+                    List<Index> indexes =
+                            MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (int k = 0; k < indexes.size(); k++) {
                         if (ExternalIndexingOperations.isFileIndex(indexes.get(k))) {
                             jobsToExecute.add(
@@ -1376,8 +1377,8 @@
     public static void doDropDataset(String dataverseName, String datasetName, MetadataProvider metadataProvider,
             boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup) throws Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
-        MutableObject<MetadataTransactionContext> mdTxnCtx = new MutableObject<>(
-                MetadataManager.INSTANCE.beginTransaction());
+        MutableObject<MetadataTransactionContext> mdTxnCtx =
+                new MutableObject<>(MetadataManager.INSTANCE.beginTransaction());
         MutableBoolean bActiveTxn = new MutableBoolean(true);
         metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
         List<JobSpecification> jobsToExecute = new ArrayList<>();
@@ -1452,8 +1453,8 @@
                 throw new AlgebricksException(
                         "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
             }
-            ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                    .getActiveNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             IActiveEntityEventsListener[] listeners = activeEventHandler.getEventListeners();
             StringBuilder builder = null;
             for (IActiveEntityEventsListener listener : listeners) {
@@ -1522,8 +1523,8 @@
                 }
                 // #. prepare a job to drop the index in NC.
                 jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds));
-                List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                        datasetName);
+                List<Index> datasetIndexes =
+                        MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                 if (datasetIndexes.size() == 2) {
                     dropFilesIndex = true;
                     // only one index + the files index, we need to delete both of the indexes
@@ -1701,12 +1702,15 @@
                 varIds.add(new VarIdentifier(v));
             }
             wrappedQuery.setExternalVars(varIds);
-            apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput);
+            apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false);
+
+            List<List<List<String>>> dependencies = FunctionUtil.getFunctionDependencies(
+                    rewriterFactory.createQueryRewriter(), cfs.getFunctionBodyExpression(), metadataProvider);
 
             Function function = new Function(dataverse, functionName, cfs.getFunctionSignature().getArity(),
                     cfs.getParamList(), Function.RETURNTYPE_VOID, cfs.getFunctionBody(),
                     rewriterFactory instanceof SqlppRewriterFactory ? Function.LANGUAGE_SQLPP : Function.LANGUAGE_AQL,
-                    FunctionKind.SCALAR.toString());
+                    FunctionKind.SCALAR.toString(), dependencies);
             MetadataManager.INSTANCE.addFunction(mdTxnCtx, function);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1778,9 +1782,9 @@
         MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
-            CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName,
-                    loadStmt.getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
-                    loadStmt.dataIsAlreadySorted());
+            CompiledLoadFromFileStatement cls =
+                    new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
+                            loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
             JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1897,8 +1901,8 @@
             throws RemoteException, AlgebricksException, ACIDException {
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, query, sessionOutput);
+        Pair<IReturningStatement, Integer> rewrittenResult =
+                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query, sessionOutput, true);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -1911,8 +1915,8 @@
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata
         // transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, insertUpsert, sessionOutput);
+        Pair<IReturningStatement, Integer> rewrittenResult =
+                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, insertUpsert, sessionOutput, true);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         String dataverseName = getActiveDataverse(rewrittenInsertUpsert.getDataverseName());
@@ -1947,8 +1951,8 @@
         MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + feedName);
         try {
-            Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName,
-                    feedName);
+            Feed feed =
+                    MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
             if (feed != null) {
                 if (cfs.getIfNotExists()) {
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1983,8 +1987,8 @@
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
-            FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE
-                    .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, policy);
+            FeedPolicyEntity feedPolicy =
+                    MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, policy);
             if (feedPolicy != null) {
                 if (cfps.getIfNotExists()) {
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2062,10 +2066,10 @@
             throws Exception {
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
         EntityId feedId = feed.getFeedId();
-        ActiveNotificationHandler activeNotificationHandler = (ActiveNotificationHandler) appCtx
-                .getActiveNotificationHandler();
-        ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeNotificationHandler
-                .getListener(feedId);
+        ActiveNotificationHandler activeNotificationHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+        ActiveEntityEventsListener listener =
+                (ActiveEntityEventsListener) activeNotificationHandler.getListener(feedId);
         if (listener != null && listener.getState() != ActivityState.STOPPED) {
             throw new AlgebricksException("Feed " + feedId
                     + " is currently active and connected to the following dataset(s) \n" + listener.toString());
@@ -2134,15 +2138,15 @@
                 String fqName = feedConnection.getDataverseName() + "." + feedConnection.getDatasetName();
                 lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), fqName);
             }
-            ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                    .getActiveNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
             if (listener == null) {
                 // Prepare policy
                 List<Dataset> datasets = new ArrayList<>();
                 for (FeedConnection connection : feedConnections) {
-                    Dataset ds = metadataProvider.findDataset(connection.getDataverseName(),
-                            connection.getDatasetName());
+                    Dataset ds =
+                            metadataProvider.findDataset(connection.getDataverseName(), connection.getDatasetName());
                     datasets.add(ds);
                 }
                 listener = new FeedEventsListener(this, metadataProvider.getApplicationContext(), hcc, entityId,
@@ -2167,8 +2171,8 @@
         String dataverseName = getActiveDataverse(sfst.getDataverseName());
         String feedName = sfst.getFeedName().getValue();
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
-        ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                .getActiveNotificationHandler();
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         // Obtain runtime info from ActiveListener
         ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
         if (listener == null) {
@@ -2194,8 +2198,8 @@
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         // TODO: Check whether we are connecting a change feed to a non-meta dataset
         // Check whether feed is alive
-        ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                .getActiveNotificationHandler();
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         // Transaction handling
         MetadataLockUtil.connectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName, dataverseName + "." + feedName);
@@ -2248,8 +2252,8 @@
         MetadataLockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName, dataverseName + "." + cfs.getFeedName());
         try {
-            ActiveNotificationHandler activeEventHandler = (ActiveNotificationHandler) appCtx
-                    .getActiveNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             // Check whether feed is alive
             ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler
                     .getListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName));
@@ -2304,8 +2308,8 @@
                 throw new AlgebricksException(
                         "Cannot compact the extrenal dataset " + datasetName + " because it has no indexes");
             }
-            Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
-                    dataverseName);
+            Dataverse dataverse =
+                    MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dataverseName);
             jobsToExecute.add(DatasetUtil.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
 
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
@@ -2435,8 +2439,8 @@
     }
 
     private void updateJobStats(JobId jobId, Stats stats) {
-        final IJobManager jobManager = ((ClusterControllerService) appCtx.getServiceContext().getControllerService())
-                .getJobManager();
+        final IJobManager jobManager =
+                ((ClusterControllerService) appCtx.getServiceContext().getControllerService()).getJobManager();
         final JobRun run = jobManager.get(jobId);
         if (run == null || run.getStatus() != JobStatus.TERMINATED) {
             return;
@@ -2804,8 +2808,8 @@
         String dataverseNameTo = getActiveDataverse(pregelixStmt.getDataverseNameTo());
         String datasetNameFrom = pregelixStmt.getDatasetNameFrom().getValue();
         String datasetNameTo = pregelixStmt.getDatasetNameTo().getValue();
-        String fullyQualifiedDatasetNameTo = DatasetUtil.isFullyQualifiedName(datasetNameTo) ? datasetNameTo
-                : dataverseNameTo + '.' + datasetNameTo;
+        String fullyQualifiedDatasetNameTo =
+                DatasetUtil.isFullyQualifiedName(datasetNameTo) ? datasetNameTo : dataverseNameTo + '.' + datasetNameTo;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(), fullyQualifiedDatasetNameTo);
@@ -2888,8 +2892,8 @@
                 throw new AlgebricksException("Tried to access non-existing dataset: " + datasetNameTo);
             }
             // Cleans up the sink dataset -- Drop and then Create.
-            DropDatasetStatement dropStmt = new DropDatasetStatement(new Identifier(dataverseNameTo),
-                    pregelixStmt.getDatasetNameTo(), true);
+            DropDatasetStatement dropStmt =
+                    new DropDatasetStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(), true);
             this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc, null);
             IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
                     toIndex.getKeyFieldSourceIndicators(), false, null);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 0b3a135..b49bd8f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -1627,7 +1627,8 @@
     public void cleanup(String testCase, List<String> badtestcases) throws Exception {
         try {
             ArrayList<String> toBeDropped = new ArrayList<>();
-            InputStream resultStream = executeQueryService("select dv.DataverseName from Metadata.`Dataverse` as dv;",
+            InputStream resultStream = executeQueryService(
+                    "select dv.DataverseName from Metadata.`Dataverse` as dv order by dv.DataverseName;",
                     getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.CLEAN_JSON);
             String out = IOUtils.toString(resultStream);
             ObjectMapper om = new ObjectMapper();
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta06/meta06.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta06/meta06.1.adm
index 58f5b77..8be6cbc 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta06/meta06.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta06/meta06.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "Name": "fun01", "Arity": "0", "Params": [  ], "ReturnType": "VOID", "Definition": "\"This is an AQL Bodied UDF\"", "Language": "AQL", "Kind": "SCALAR" }
+{ "DataverseName": "testdv", "Name": "fun01", "Arity": "0", "Params": [  ], "ReturnType": "VOID", "Definition": "\"This is an AQL Bodied UDF\"", "Language": "AQL", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index d92fcd2..822d2c3 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,66 +1,69 @@
-{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Dec 08 08:42:06 UTC 2017" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Mon Jan 08 10:27:05 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false }, { "FieldName": "Dependencies", "FieldType": "FunctionRecordType_Dependencies", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.1.ddl.sqlpp
new file mode 100644
index 0000000..c9639c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.1.ddl.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Verify Function Dependency Metadata
+ * Expected Res : Success
+ * Date         : Dec 15th 2017
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse B;
+create dataverse C;
+use C;
+
+create type TweetMessageType as closed {
+  tweetid: uuid,
+  sender_location: point,
+  send_time: datetime,
+  referred_topics: {{ string }},
+  message_text: string,
+  countA: int32,
+  countB: int32
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid autogenerated;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+create function f4(){
+(select * from TweetMessages)
+};
+
+use B;
+
+create dataset TweetMessages2(C.TweetMessageType)
+primary key tweetid autogenerated;
+
+create function f0(message, text){
+  contains(message,text)
+};
+
+create function C.f2(place, text) {
+  (select m.message_text
+  from TweetMessages m
+  where contains(m.message_text,text)
+  and spatial_intersect(m.sender_location, place)
+  and f1(m.message_text,text)
+  and B.f0(m.message_text,text))
+};
+
+create function C.f3(place, text) {
+  f2(place, text)
+};
+
+create function f5(place, text){
+ (select m.message_text
+  from TweetMessages2 m, C.TweetMessages m2
+  where contains(m.message_text,text)
+  and spatial_intersect(m.sender_location, place)
+  and C.f1(m.message_text,text)
+  and f0(m2.message_text,text))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.2.query.sqlpp
new file mode 100644
index 0000000..e9ab11e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Verify Function Dependency Metadata
+ * Expected Res : Success
+ * Date         : Dec 15th 2017
+ */
+
+select f.DataverseName,f.Name,f.Dependencies from Metadata.`Function` f
+order by f.DataverseName, f.Name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp
new file mode 100644
index 0000000..215118b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Verify Function Dependency Metadata
+ * Expected Res : Success
+ * Date         : Dec 15th 2017
+ */
+
+drop function B.f5@2;
+drop function C.f3@2;
+drop function C.f2@2;
+
+drop dataset B.TweetMessages2;
+
+drop dataverse C;
+drop dataverse B;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-1/drop-dependency.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-1/drop-dependency.1.ddl.sqlpp
new file mode 100644
index 0000000..08ad454
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-1/drop-dependency.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse B;
+create dataverse C;
+use C;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+use B;
+
+create function f0(message, text){
+  C.f1(message,text)
+};
+
+drop dataverse C;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-2/drop-dependency.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-2/drop-dependency.2.ddl.sqlpp
new file mode 100644
index 0000000..53debdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-2/drop-dependency.2.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse B;
+create dataverse C;
+use C;
+
+create type TweetMessageType as closed {
+  tweetid: uuid,
+  sender_location: point,
+  send_time: datetime,
+  referred_topics: {{ string }},
+  message_text: string,
+  countA: int32,
+  countB: int32
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid autogenerated;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+use B;
+
+create function f2(place, text){
+ (select m.message_text
+  from C.TweetMessages m)
+};
+
+drop dataverse C;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-3/drop-dependency.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-3/drop-dependency.3.ddl.sqlpp
new file mode 100644
index 0000000..7e7b60e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-3/drop-dependency.3.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse B;
+create dataverse C;
+use C;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+use B;
+
+create function f0(message, text){
+  C.f1(message,text)
+};
+
+use C;
+drop function f1@2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-4/drop-dependency.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-4/drop-dependency.4.ddl.sqlpp
new file mode 100644
index 0000000..c972efc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-4/drop-dependency.4.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse B;
+create dataverse C;
+use C;
+
+create type TweetMessageType as closed {
+  tweetid: uuid,
+  sender_location: point,
+  send_time: datetime,
+  referred_topics: {{ string }},
+  message_text: string,
+  countA: int32,
+  countB: int32
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid autogenerated;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+use B;
+
+create function f2(place, text){
+ (select m.message_text
+  from C.TweetMessages m)
+};
+
+use C;
+drop dataset TweetMessages;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-5/drop-dependency.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-5/drop-dependency.5.ddl.sqlpp
new file mode 100644
index 0000000..3a7f4dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-5/drop-dependency.5.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse C;
+use C;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+create function f0(message, text){
+  C.f1(message,text)
+};
+
+drop function f1@2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-6/drop-dependency.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-6/drop-dependency.6.ddl.sqlpp
new file mode 100644
index 0000000..3ab7b6f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/drop-dependency-6/drop-dependency.6.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under A
+ * 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  : Try to drop a functional dependency
+ * Expected Res : Error
+ */
+
+drop dataverse B if exists;
+drop dataverse C if exists;
+create dataverse C;
+use C;
+
+create type TweetMessageType as closed {
+  tweetid: uuid,
+  sender_location: point,
+  send_time: datetime,
+  referred_topics: {{ string }},
+  message_text: string,
+  countA: int32,
+  countB: int32
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid autogenerated;
+
+create function f1(message, text){
+  contains(message,text)
+};
+
+create function f2(place, text){
+ (select m.message_text
+  from C.TweetMessages m)
+};
+
+drop dataset TweetMessages;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
index 8f4c581..403237e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
@@ -1,8 +1,8 @@
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#addHashTags", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#addHashTagsInPlace", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsInPlaceFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "org.apache.asterix.external.library.AllTypesFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "org.apache.asterix.external.library.EchoDelayFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "org.apache.asterix.external.library.CapitalFinderFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "org.apache.asterix.external.library.SumFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR" } }
-{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR" } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#addHashTags", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#addHashTagsInPlace", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsInPlaceFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "org.apache.asterix.external.library.AllTypesFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "org.apache.asterix.external.library.EchoDelayFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "org.apache.asterix.external.library.CapitalFinderFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "org.apache.asterix.external.library.SumFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm
new file mode 100644
index 0000000..50f19bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm
@@ -0,0 +1,6 @@
+{ "DataverseName": "B", "Name": "f0", "Dependencies": [ [  ], [  ] ] }
+{ "DataverseName": "B", "Name": "f5", "Dependencies": [ [ [ "C", "TweetMessages" ], [ "B", "TweetMessages2" ] ], [ [ "C", "f1", "2" ], [ "B", "f0", "2" ] ] ] }
+{ "DataverseName": "C", "Name": "f1", "Dependencies": [ [  ], [  ] ] }
+{ "DataverseName": "C", "Name": "f2", "Dependencies": [ [ [ "C", "TweetMessages" ] ], [ [ "C", "f1", "2" ], [ "B", "f0", "2" ] ] ] }
+{ "DataverseName": "C", "Name": "f3", "Dependencies": [ [  ], [ [ "C", "f2", "2" ] ] ] }
+{ "DataverseName": "C", "Name": "f4", "Dependencies": [ [ [ "C", "TweetMessages" ] ], [  ] ] }
\ 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 fd47766..14784d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -7922,7 +7922,7 @@
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="bad-function-ddl-9">
         <output-dir compare="Text">bad-function-ddl-9</output-dir>
-        <expected-error>function experiments.good_function@1 depends upon function experiments.function_that_does_not_exist@0 which is undefined</expected-error>
+        <expected-error>function experiments.function_that_does_not_exist@0 is not defined</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
@@ -7932,6 +7932,47 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="check-dependencies-1">
+        <output-dir compare="Text">check-dependencies-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-1">
+        <output-dir compare="Text">drop-dependency-1</output-dir>
+        <expected-error>Cannot drop dataverse. Function B.f0@2 depends on function C.f1@2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-2">
+        <output-dir compare="Text">drop-dependency-2</output-dir>
+        <expected-error>Cannot drop dataverse. Function B.f2@2 depends on dataset C.TweetMessages</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-3">
+        <output-dir compare="Text">drop-dependency-3</output-dir>
+        <expected-error>Cannot drop function C.f1@2 being used by function B.f0@2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-4">
+        <output-dir compare="Text">drop-dependency-4</output-dir>
+        <expected-error>Cannot drop dataset C.TweetMessages being used by function B.f2@2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-5">
+        <output-dir compare="Text">drop-dependency-5</output-dir>
+        <expected-error>Cannot drop function C.f1@2 being used by function C.f0@2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-6">
+        <output-dir compare="Text">drop-dependency-6</output-dir>
+        <expected-error>Cannot drop dataset C.TweetMessages being used by function C.f2@2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="single-line-definition">
         <output-dir compare="Text">single-line-definition</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
index 977f6bb..6ab69ad 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -68,7 +69,8 @@
 
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context) throws CompilationException {
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
+            throws CompilationException {
         setup(declaredFunctions, topStatement, metadataProvider, context);
         if (topStatement.isTopLevel()) {
             wrapInLets();
@@ -130,7 +132,8 @@
         declaredFunctions.removeAll(storedFunctionDecls);
     }
 
-    private Set<FunctionSignature> getFunctionCalls(Expression expression) throws CompilationException {
+    @Override
+    public Set<CallExpr> getFunctionCalls(Expression expression) throws CompilationException {
         GatherFunctionCalls gfc = new GatherFunctionCalls();
         expression.accept(gfc, null);
         return gfc.getCalls();
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 0c6c04c..f072917 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,8 +19,10 @@
 package org.apache.asterix.lang.common.base;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -39,6 +41,13 @@
      * @param context,
      *            manages ids of variables and guarantees uniqueness of variables.
      */
-    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context) throws CompilationException;
+    void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
+            throws CompilationException;
+
+    /**
+     * Find the function calls used by a given expression
+     */
+    Set<CallExpr> getFunctionCalls(Expression expression) throws CompilationException;
+
 }
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 1ca9316..3571dcf 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
@@ -20,6 +20,7 @@
 package org.apache.asterix.lang.common.util;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
@@ -27,13 +28,18 @@
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 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.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
@@ -52,7 +58,7 @@
 
     @FunctionalInterface
     public interface IFunctionCollector {
-        Set<FunctionSignature> getFunctionCalls(Expression expression) throws CompilationException;
+        Set<CallExpr> getFunctionCalls(Expression expression) throws CompilationException;
     }
 
     @FunctionalInterface
@@ -95,8 +101,9 @@
         }
         String value = metadataProvider.getConfig().get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS);
         boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
-        Set<FunctionSignature> functionCalls = functionCollector.getFunctionCalls(expression);
-        for (FunctionSignature signature : functionCalls) {
+        Set<CallExpr> functionCalls = functionCollector.getFunctionCalls(expression);
+        for (CallExpr functionCall : functionCalls) {
+            FunctionSignature signature = functionCall.getFunctionSignature();
             if (declaredFunctions != null && declaredFunctions.contains(signature)) {
                 continue;
             }
@@ -155,6 +162,35 @@
         return functionDecls;
     }
 
+    public static List<List<List<String>>> getFunctionDependencies(IQueryRewriter rewriter, Expression expression,
+            MetadataProvider metadataProvider) throws CompilationException {
+        Set<CallExpr> functionCalls = rewriter.getFunctionCalls(expression);
+        //Get the List of used functions and used datasets
+        List<List<String>> datasourceDependencies = new ArrayList<>();
+        List<List<String>> functionDependencies = new ArrayList<>();
+        for (CallExpr functionCall : functionCalls) {
+            FunctionSignature signature = functionCall.getFunctionSignature();
+            FunctionIdentifier fid =
+                    new FunctionIdentifier(signature.getNamespace(), signature.getName(), signature.getArity());
+            if (fid.equals(BuiltinFunctions.DATASET)) {
+                Pair<String, String> path = DatasetUtil.getDatasetInfo(metadataProvider,
+                        ((LiteralExpr) functionCall.getExprList().get(0)).getValue().getStringValue());
+                datasourceDependencies.add(Arrays.asList(path.first, path.second));
+            }
+
+            else if (BuiltinFunctions.isBuiltinCompilerFunction(signature, false)) {
+                continue;
+            } else {
+                functionDependencies.add(Arrays.asList(signature.getNamespace(), signature.getName(),
+                        Integer.toString(signature.getArity())));
+            }
+        }
+        List<List<List<String>>> dependencies = new ArrayList<>();
+        dependencies.add(datasourceDependencies);
+        dependencies.add(functionDependencies);
+        return dependencies;
+    }
+
     private static Function lookupUserDefinedFunctionDecl(MetadataTransactionContext mdTxnCtx,
             FunctionSignature signature) throws AlgebricksException {
         if (signature.getNamespace() == null) {
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 7d8c11e..8ae67d6 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
@@ -347,7 +347,7 @@
         metadataProvider.setDefaultDataverse(fnDataverse);
         try {
             IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
-            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context);
+            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context, true);
             return wrappedQuery.getBody();
         } finally {
             metadataProvider.setDefaultDataverse(defaultDataverse);
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 8842d86..3d149fb 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
@@ -23,7 +23,6 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -53,11 +52,11 @@
 
 public class GatherFunctionCallsVisitor extends AbstractQueryExpressionVisitor<Void, Void> {
 
-    protected final Set<FunctionSignature> calls = new HashSet<FunctionSignature>();
+    protected final Set<CallExpr> calls = new HashSet<>();
 
     @Override
     public Void visit(CallExpr pf, Void arg) throws CompilationException {
-        calls.add(pf.getFunctionSignature());
+        calls.add(pf);
         for (Expression e : pf.getExprList()) {
             e.accept(this, arg);
         }
@@ -202,7 +201,7 @@
         return null;
     }
 
-    public Set<FunctionSignature> getCalls() {
+    public Set<CallExpr> getCalls() {
         return calls;
     }
 
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 a8b3dc6..cd57396 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
@@ -31,7 +31,7 @@
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
             MetadataProvider metadataProvider,
-            LangRewritingContext context) throws CompilationException {
+            LangRewritingContext context, boolean inlineUdfs) throws CompilationException {
         // Sets up parameters.
         setup(declaredFunctions, topStatement, metadataProvider, context);
 
@@ -63,6 +63,6 @@
         rewriteListInputFunctions();
 
         // Inlines functions recursively.
-        inlineDeclaredUdfs();
+        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 3e9a873..befa5ab 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
@@ -28,6 +28,7 @@
 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.CallExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.util.FunctionUtil;
@@ -86,7 +87,8 @@
 
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context) throws CompilationException {
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
+            throws CompilationException {
         if (topStatement == null) {
             return;
         }
@@ -126,7 +128,7 @@
         rewriteListInputFunctions();
 
         // Inlines functions.
-        inlineDeclaredUdfs();
+        inlineDeclaredUdfs(inlineUdfs);
 
         // Rewrites function names.
         // This should be done after inlineDeclaredUdfs() because user-defined function
@@ -213,7 +215,7 @@
         topExpr.accept(groupByVisitor, null);
     }
 
-    protected void inlineDeclaredUdfs() throws CompilationException {
+    protected void inlineDeclaredUdfs(boolean inlineUdfs) throws CompilationException {
         List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
         for (FunctionDecl fdecl : declaredFunctions) {
             funIds.add(fdecl.getSignature());
@@ -226,7 +228,7 @@
                     signature -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false)));
         }
         declaredFunctions.addAll(usedStoredFunctionDecls);
-        if (!declaredFunctions.isEmpty()) {
+        if (inlineUdfs && !declaredFunctions.isEmpty()) {
             SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context,
                     new SqlppFunctionBodyRewriterFactory() /* the rewriter for function bodies expressions*/,
                     declaredFunctions, metadataProvider);
@@ -237,7 +239,8 @@
         declaredFunctions.removeAll(usedStoredFunctionDecls);
     }
 
-    private Set<FunctionSignature> getFunctionCalls(Expression expression) throws CompilationException {
+    @Override
+    public Set<CallExpr> getFunctionCalls(Expression expression) throws CompilationException {
         GatherFunctionCalls gfc = new GatherFunctionCalls();
         expression.accept(gfc, null);
         return gfc.getCalls();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index ed3bbe0..6634e51 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -268,7 +268,6 @@
         }
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends IExtensionMetadataEntity> void addEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
@@ -280,7 +279,6 @@
         addEntity(txnId, entity, tupleTranslator, index);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends IExtensionMetadataEntity> void upsertEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
@@ -292,7 +290,6 @@
         upsertEntity(txnId, entity, tupleTranslator, index);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends IExtensionMetadataEntity> void deleteEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
@@ -304,7 +301,6 @@
         deleteEntity(txnId, entity, tupleTranslator, index);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException {
@@ -527,13 +523,22 @@
 
             confirmDataverseCanBeDeleted(txnId, dataverseName);
 
+            // As a side effect, acquires an S lock on the 'Function' dataset
+            // on behalf of txnId.
+            List<Function> dataverseFunctions = getDataverseFunctions(txnId, dataverseName);
+            // Drop all functions in this dataverse.
+            for (Function function : dataverseFunctions) {
+                dropFunction(txnId, new FunctionSignature(dataverseName, function.getName(), function.getArity()),
+                        true);
+            }
+
             List<Dataset> dataverseDatasets;
             Dataset ds;
             dataverseDatasets = getDataverseDatasets(txnId, dataverseName);
             // Drop all datasets in this dataverse.
             for (int i = 0; i < dataverseDatasets.size(); i++) {
                 ds = dataverseDatasets.get(i);
-                dropDataset(txnId, dataverseName, ds.getDatasetName());
+                dropDataset(txnId, dataverseName, ds.getDatasetName(), true);
             }
 
             // After dropping datasets, drop datatypes
@@ -546,14 +551,6 @@
                 forceDropDatatype(txnId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
             }
 
-            // As a side effect, acquires an S lock on the 'Function' dataset
-            // on behalf of txnId.
-            List<Function> dataverseFunctions = getDataverseFunctions(txnId, dataverseName);
-            // Drop all functions in this dataverse.
-            for (Function function : dataverseFunctions) {
-                dropFunction(txnId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
-            }
-
             // As a side effect, acquires an S lock on the 'Adapter' dataset
             // on behalf of txnId.
             List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(txnId, dataverseName);
@@ -609,6 +606,16 @@
     @Override
     public void dropDataset(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException {
+        dropDataset(txnId, dataverseName, datasetName, false);
+    }
+
+    public void dropDataset(TxnId txnId, String dataverseName, String datasetName, boolean force)
+            throws AlgebricksException, RemoteException {
+
+        if (!force) {
+            confirmDatasetCanBeDeleted(txnId, dataverseName, datasetName);
+        }
+
         Dataset dataset = getDataset(txnId, dataverseName, datasetName);
         if (dataset == null) {
             throw new AlgebricksException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
@@ -902,6 +909,19 @@
         }
     }
 
+    public List<Function> getAllFunctions(TxnId txnId) throws AlgebricksException, RemoteException {
+        try {
+            ITupleReference searchKey = null;
+            FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
+            IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+            List<Function> results = new ArrayList<>();
+            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            return results;
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     public List<Datatype> getAllDatatypes(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = null;
@@ -932,6 +952,62 @@
                                 + " used by dataset " + set.getDataverseName() + "." + set.getDatasetName());
             }
         }
+
+        // If a function from a DIFFERENT dataverse
+        // uses functions or datatypes from this dataverse
+        // throw an error
+        List<Function> functions = getAllFunctions(txnId);
+        for (Function function : functions) {
+            if (function.getDataverseName().equals(dataverseName)) {
+                continue;
+            }
+            for (List<String> datasetDependency : function.getDependencies().get(0)) {
+                if (datasetDependency.get(0).equals(dataverseName)) {
+                    throw new AlgebricksException("Cannot drop dataverse. Function " + function.getDataverseName() + "."
+                            + function.getName() + "@" + function.getArity() + " depends on dataset "
+                            + datasetDependency.get(0) + "." + datasetDependency.get(1));
+                }
+            }
+            for (List<String> functionDependency : function.getDependencies().get(1)) {
+                if (functionDependency.get(0).equals(dataverseName)) {
+                    throw new AlgebricksException(
+                            "Cannot drop dataverse. Function " + function.getDataverseName() + "." + function.getName()
+                                    + "@" + function.getArity() + " depends on function " + functionDependency.get(0)
+                                    + "." + functionDependency.get(1) + "@" + functionDependency.get(2));
+                }
+            }
+        }
+    }
+
+    private void confirmFunctionCanBeDeleted(TxnId txnId, FunctionSignature signature)
+            throws AlgebricksException, RemoteException {
+        // If any other function uses this function, throw an error
+        List<Function> functions = getAllFunctions(txnId);
+        for (Function function : functions) {
+            for (List<String> functionalDependency : function.getDependencies().get(1)) {
+                if (functionalDependency.get(0).equals(signature.getNamespace())
+                        && functionalDependency.get(1).equals(signature.getName())
+                        && functionalDependency.get(2).equals(Integer.toString(signature.getArity()))) {
+                    throw new AlgebricksException("Cannot drop function " + signature + " being used by function "
+                            + function.getDataverseName() + "." + function.getName() + "@" + function.getArity());
+                }
+            }
+        }
+    }
+
+    private void confirmDatasetCanBeDeleted(TxnId txnId, String dataverseName, String datasetName)
+            throws AlgebricksException, RemoteException {
+        // If any function uses this type, throw an error
+        List<Function> functions = getAllFunctions(txnId);
+        for (Function function : functions) {
+            for (List<String> datasetDependency : function.getDependencies().get(0)) {
+                if (datasetDependency.get(0).equals(dataverseName) && datasetDependency.get(1).equals(datasetName)) {
+                    throw new AlgebricksException("Cannot drop dataset " + dataverseName + "." + datasetName
+                            + " being used by function " + function.getDataverseName() + "." + function.getName() + "@"
+                            + function.getArity());
+                }
+            }
+        }
     }
 
     private void confirmDatatypeIsUnused(TxnId txnId, String dataverseName, String datatypeName)
@@ -1123,6 +1199,15 @@
     @Override
     public void dropFunction(TxnId txnId, FunctionSignature functionSignature)
             throws AlgebricksException, RemoteException {
+        dropFunction(txnId, functionSignature, false);
+    }
+
+    private void dropFunction(TxnId txnId, FunctionSignature functionSignature, boolean force)
+            throws AlgebricksException, RemoteException {
+
+        if (!force) {
+            confirmFunctionCanBeDeleted(txnId, functionSignature);
+        }
 
         Function function = getFunction(txnId, functionSignature);
 
@@ -1329,7 +1414,6 @@
     // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
     // Hyracks version.
     public static ITupleReference createTuple(String... fields) {
-        @SuppressWarnings("unchecked")
         ISerializerDeserializer<AString> stringSerde =
                 SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
         AMutableString aString = new AMutableString("");
@@ -1836,7 +1920,6 @@
 
     // This method is used to create a search tuple for external data file since the
     // search tuple has an int value
-    @SuppressWarnings("unchecked")
     public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber)
             throws HyracksDataException {
         ISerializerDeserializer<AString> stringSerde =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 841065b..cb67ee5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -158,7 +158,7 @@
 
     public void dropFunction(FunctionSignature signature) {
         Function function = new Function(signature.getNamespace(), signature.getName(), signature.getArity(), null,
-                null, null, null, null);
+                null, null, null, null, null);
         droppedCache.addFunctionIfNotExists(function);
         logAndApply(new MetadataLogicalOperation(function, false));
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index e0f925c..e2724e2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -48,6 +48,7 @@
     public static final String FIELD_NAME_DATAVERSE_NAME = "DataverseName";
     public static final String FIELD_NAME_DATA_FORMAT = "DataFormat";
     public static final String FIELD_NAME_DEFINITION = "Definition";
+    public static final String FIELD_NAME_DEPENDENCIES = "Dependencies";
     public static final String FIELD_NAME_DERIVED = "Derived";
     public static final String FIELD_NAME_DESCRIPTION = "Description";
     public static final String FIELD_NAME_EXTERNAL_DETAILS = "ExternalDetails";
@@ -321,16 +322,20 @@
     public static final int FUNCTION_ARECORD_FUNCTION_DEFINITION_FIELD_INDEX = 5;
     public static final int FUNCTION_ARECORD_FUNCTION_LANGUAGE_FIELD_INDEX = 6;
     public static final int FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX = 7;
+    public static final int FUNCTION_ARECORD_FUNCTION_DEPENDENCIES_FIELD_INDEX = 8;
     public static final ARecordType FUNCTION_RECORDTYPE = createRecordType(
             // RecordTypeName
             RECORD_NAME_FUNCTION,
             // FieldNames
             new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_NAME, FIELD_NAME_ARITY, FIELD_NAME_PARAMS,
-                    FIELD_NAME_RETURN_TYPE, FIELD_NAME_DEFINITION, FIELD_NAME_LANGUAGE, FIELD_NAME_KIND },
+                    FIELD_NAME_RETURN_TYPE, FIELD_NAME_DEFINITION, FIELD_NAME_LANGUAGE, FIELD_NAME_KIND,
+                    FIELD_NAME_DEPENDENCIES },
             // FieldTypes
             new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                     new AOrderedListType(BuiltinType.ASTRING, null), BuiltinType.ASTRING, BuiltinType.ASTRING,
-                    BuiltinType.ASTRING, BuiltinType.ASTRING },
+                    BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                            null) },
             //IsOpen?
             true);
     //------------------------------------------ Adapter ----------------------------------------//
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
index 1d1db37..8bdbaba 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.metadata.entities;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.metadata.MetadataCache;
@@ -36,13 +37,14 @@
     private final String name;
     private final int arity;
     private final List<String> params;
+    private final List<List<List<String>>> dependencies;
     private final String body;
     private final String returnType;
     private final String language;
     private final String kind;
 
     public Function(String dataverseName, String functionName, int arity, List<String> params, String returnType,
-            String functionBody, String language, String functionKind) {
+            String functionBody, String language, String functionKind, List<List<List<String>>> dependencies) {
         this.dataverse = dataverseName;
         this.name = functionName;
         this.params = params;
@@ -51,6 +53,13 @@
         this.language = language;
         this.kind = functionKind;
         this.arity = arity;
+        if (dependencies == null) {
+            this.dependencies = new ArrayList<>();
+            this.dependencies.add(new ArrayList<>());
+            this.dependencies.add(new ArrayList<>());
+        } else {
+            this.dependencies = dependencies;
+        }
     }
 
     public String getDataverseName() {
@@ -65,6 +74,10 @@
         return params;
     }
 
+    public List<List<List<String>>> getDependencies() {
+        return dependencies;
+    }
+
     public String getFunctionBody() {
         return body;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index e15805e..273c78a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -35,6 +35,7 @@
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -56,7 +57,13 @@
     // Payload field containing serialized Function.
     public static final int FUNCTION_PAYLOAD_TUPLE_FIELD_INDEX = 3;
 
-    @SuppressWarnings("unchecked")
+    private transient OrderedListBuilder dependenciesListBuilder = new OrderedListBuilder();
+    private transient OrderedListBuilder dependencyListBuilder = new OrderedListBuilder();
+    private transient OrderedListBuilder dependencyNameListBuilder = new OrderedListBuilder();
+    private transient AOrderedListType stringList = new AOrderedListType(BuiltinType.ASTRING, null);
+    private transient AOrderedListType ListofLists =
+            new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null);
+
     private ISerializerDeserializer<ARecord> recordSerDes =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.FUNCTION_RECORDTYPE);
 
@@ -104,8 +111,33 @@
         String functionKind =
                 ((AString) functionRecord.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX))
                         .getStringValue();
+
+        IACursor dependenciesCursor = ((AOrderedList) functionRecord
+                .getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_DEPENDENCIES_FIELD_INDEX)).getCursor();
+        List<List<List<String>>> dependencies = new ArrayList<>();
+        AOrderedList dependencyList;
+        AOrderedList qualifiedList;
+        int i = 0;
+        while (dependenciesCursor.next()) {
+            dependencies.add(new ArrayList<>());
+            dependencyList = (AOrderedList) dependenciesCursor.get();
+            IACursor qualifiedDependencyCursor = dependencyList.getCursor();
+            int j = 0;
+            while (qualifiedDependencyCursor.next()) {
+                qualifiedList = (AOrderedList) qualifiedDependencyCursor.get();
+                IACursor qualifiedNameCursor = qualifiedList.getCursor();
+                dependencies.get(i).add(new ArrayList<>());
+                while (qualifiedNameCursor.next()) {
+                    dependencies.get(i).get(j).add(((AString) qualifiedNameCursor.get()).getStringValue());
+                }
+                j++;
+            }
+            i++;
+
+        }
+
         return new Function(dataverseName, functionName, Integer.parseInt(arity), params, returnType, definition,
-                language, functionKind);
+                language, functionKind, dependencies);
 
     }
 
@@ -185,6 +217,33 @@
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX, fieldValue);
 
+        // write field 8
+        dependenciesListBuilder.reset((AOrderedListType) MetadataRecordTypes.FUNCTION_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_DEPENDENCIES_FIELD_INDEX]);
+        List<List<List<String>>> dependenciesList = function.getDependencies();
+        for (List<List<String>> dependencies : dependenciesList) {
+            dependencyListBuilder.reset(ListofLists);
+            for (List<String> dependency : dependencies) {
+                dependencyNameListBuilder.reset(stringList);
+                for (String subName : dependency) {
+                    itemValue.reset();
+                    aString.setValue(subName);
+                    stringSerde.serialize(aString, itemValue.getDataOutput());
+                    dependencyNameListBuilder.addItem(itemValue);
+                }
+                itemValue.reset();
+                dependencyNameListBuilder.write(itemValue.getDataOutput(), true);
+                dependencyListBuilder.addItem(itemValue);
+
+            }
+            itemValue.reset();
+            dependencyListBuilder.write(itemValue.getDataOutput(), true);
+            dependenciesListBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        dependenciesListBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_DEPENDENCIES_FIELD_INDEX, fieldValue);
+
         // write record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();