merge from master
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index ab3516a..c3c50b4 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -47,6 +47,7 @@
 import edu.uci.ics.asterix.optimizer.rules.PushProperJoinThroughProduct;
 import edu.uci.ics.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
 import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
+import edu.uci.ics.asterix.optimizer.rules.RemoveSortInFeedIngestionRule;
 import edu.uci.ics.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.ReplaceSinkOpWithCommitOpRule;
 import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
@@ -239,6 +240,7 @@
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
+        physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
         physicalRewritesAllLevels.add(new IntroHashPartitionMergeExchange());
         physicalRewritesAllLevels.add(new SetClosedRecordConstructorsRule());
         physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
index 62cca6c..a2b8ef5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataImplConfig;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -81,11 +82,11 @@
                 }
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
                 DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
-                String datasetName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getDataset().getDatasetName();
-                if (dataSourcesMap.containsKey(datasetName)) {
-                    ++(dataSourcesMap.get(datasetName).first);
+                String datasourceName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getDatasourceName();
+                if (dataSourcesMap.containsKey(datasourceName)) {
+                    ++(dataSourcesMap.get(datasourceName).first);
                 } else {
-                    dataSourcesMap.put(datasetName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
+                    dataSourcesMap.put(datasourceName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
                             LogicalOperatorTag.DATASOURCESCAN, dataSourceScanOp.getPhysicalOperator()));
                 }
             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index d33d674..cb10363 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -133,7 +133,7 @@
         if (dataset == null) {
             throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             return false;
         }
         ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
@@ -308,8 +308,7 @@
                             if (dataset == null) {
                                 throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
                             }
-                            if (dataset.getDatasetType() != DatasetType.INTERNAL
-                                    && dataset.getDatasetType() != DatasetType.FEED) {
+                            if (dataset.getDatasetType() != DatasetType.INTERNAL) {
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
                             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java
new file mode 100644
index 0000000..951cacf
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class RemoveSortInFeedIngestionRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE) {
+            return false;
+        }
+
+        AbstractLogicalOperator insertOp = op;
+        AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        boolean isSourceAFeed = false;
+        while (descendantOp != null) {
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                AqlDataSource dataSource = (AqlDataSource) ((DataSourceScanOperator) descendantOp).getDataSource();
+                if (dataSource.getDatasourceType().equals(AqlDataSourceType.FEED)) {
+                    isSourceAFeed = true;
+                }
+                break;
+            }
+            if (descendantOp.getInputs().isEmpty()) {
+                break;
+            }
+            descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+        }
+
+        if (isSourceAFeed) {
+            AbstractLogicalOperator prevOp = (AbstractLogicalOperator) insertOp.getInputs().get(0).getValue();
+            if (prevOp.getOperatorTag() == LogicalOperatorTag.ORDER) {
+                insertOp.getInputs().set(0, prevOp.getInputs().get(0));
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
index 9fe5e46..09715f5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -23,6 +23,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -45,14 +46,13 @@
  * Removes join operators for which all of the following conditions are true:
  * 1. The live variables of one input branch of the join are not used in the upstream plan
  * 2. The join is an inner equi join
- * 3. The join condition only uses variables that correspond to primary keys of the same dataset    
+ * 3. The join condition only uses variables that correspond to primary keys of the same dataset
  * Notice that the last condition implies a 1:1 join, i.e., the join does not change the result cardinality.
- * 
- * Joins that satisfy the above conditions may be introduced by other rules 
+ * Joins that satisfy the above conditions may be introduced by other rules
  * which use surrogate optimizations. Such an optimization aims to reduce data copies and communication costs by
  * using the primary keys as surrogates for the desired data items. Typically,
  * such a surrogate-based plan introduces a top-level join to finally resolve
- * the surrogates to the desired data items. 
+ * the surrogates to the desired data items.
  * In case the upstream plan does not require the original data items at all, such a top-level join is unnecessary.
  * The purpose of this rule is to remove such unnecessary joins.
  */
@@ -152,8 +152,8 @@
         // only used primary key variables of those datascans.
         for (int i = 0; i < dataScans.size(); i++) {
             if (i > 0) {
-                AqlDataSource prevAqlDataSource = (AqlDataSource) dataScans.get(i - 1).getDataSource();
-                AqlDataSource currAqlDataSource = (AqlDataSource) dataScans.get(i).getDataSource();
+                DatasetDataSource prevAqlDataSource = (DatasetDataSource) dataScans.get(i - 1).getDataSource();
+                DatasetDataSource currAqlDataSource = (DatasetDataSource) dataScans.get(i).getDataSource();
                 if (!prevAqlDataSource.getDataset().equals(currAqlDataSource.getDataset())) {
                     return -1;
                 }
@@ -189,8 +189,8 @@
 
     private void fillPKVars(DataSourceScanOperator dataScan, List<LogicalVariable> pkVars) {
         pkVars.clear();
-        AqlDataSource aqlDataSource = (AqlDataSource) dataScan.getDataSource();
-        int numPKs = DatasetUtils.getPartitioningKeys(aqlDataSource.getDataset()).size();
+        DatasetDataSource datasetDataSource = (DatasetDataSource) dataScan.getDataSource();
+        int numPKs = DatasetUtils.getPartitioningKeys(datasetDataSource.getDataset()).size();
         pkVars.clear();
         for (int i = 0; i < numPKs; i++) {
             pkVars.add(dataScan.getVariables().get(i));
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index 1fe7ee9..2550a34 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -63,12 +64,13 @@
             if (descendantOp.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE) {
                 IndexInsertDeleteOperator indexInsertDeleteOperator = (IndexInsertDeleteOperator) descendantOp;
                 primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
-                datasetId = ((AqlDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource()).getDataset().getDatasetId();
+                datasetId = ((DatasetDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource())
+                        .getDataset().getDatasetId();
                 break;
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
                 InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) descendantOp;
                 primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
-                datasetId = ((AqlDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+                datasetId = ((DatasetDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
                 break;
             }
             descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
@@ -79,7 +81,7 @@
             //copy primaryKeyExprs
             List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
             for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
-                VariableReferenceExpression varRefExpr = (VariableReferenceExpression)expr.getValue();
+                VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr.getValue();
                 primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
             }
 
@@ -89,13 +91,14 @@
 
             //create the logical and physical operator
             CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
-            CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, mp.isWriteTransaction());
+            CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars,
+                    mp.isWriteTransaction());
             commitOperator.setPhysicalOperator(commitPOperator);
 
             //create ExtensionOperator and put the commitOperator in it.
             ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
             extensionOperator.setPhysicalOperator(commitPOperator);
-            
+
             //update plan link
             extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
             context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
index 091d1c1..a0cf4f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -23,11 +23,13 @@
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.metadata.declared.ExternalFeedDataSource;
+import edu.uci.ics.asterix.metadata.declared.FeedDataSource;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
 import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -106,7 +108,7 @@
 
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
 
-                if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+                if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                     int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
                     for (int i = 0; i < numPrimaryKeys; i++) {
                         v.add(context.newVar());
@@ -127,35 +129,21 @@
                 if (unnest.getPositionalVariable() != null) {
                     throw new AlgebricksException("No positional variables are allowed over datasets.");
                 }
-                ILogicalExpression expr = f.getArguments().get(0).getValue();
-                if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                    return false;
-                }
-                ConstantExpression ce = (ConstantExpression) expr;
-                IAlgebricksConstantValue acv = ce.getValue();
-                if (!(acv instanceof AsterixConstantValue)) {
-                    return false;
-                }
-                AsterixConstantValue acv2 = (AsterixConstantValue) acv;
-                if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
-                    return false;
-                }
-                String datasetArg = ((AString) acv2.getObject()).getStringValue();
+
+                String feedArg = getStringArgument(f, 0);
+                String outputType = getStringArgument(f, 1);
+                String targetDataset = getStringArgument(f, 2);
 
                 AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
-                Pair<String, String> datasetReference = parseDatasetReference(metadataProvider, datasetArg);
-                String dataverseName = datasetReference.first;
-                String datasetName = datasetReference.second;
-                Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-                if (dataset == null) {
-                    throw new AlgebricksException("Could not find dataset " + datasetName);
+                Pair<String, String> feedReference = parseDatasetReference(metadataProvider, feedArg);
+                String dataverseName = feedReference.first;
+                String feedName = feedReference.second;
+                Feed feed = metadataProvider.findFeed(dataverseName, feedName);
+                if (feed == null) {
+                    throw new AlgebricksException("Could not find feed " + feedName);
                 }
 
-                if (dataset.getDatasetType() != DatasetType.FEED) {
-                    throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
-                }
-
-                AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
+                AqlSourceId asid = new AqlSourceId(dataverseName, feedName);
                 String policyName = metadataProvider.getConfig().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
                 FeedPolicy policy = metadataProvider.findFeedPolicy(dataverseName, policyName);
                 if (policy == null) {
@@ -168,8 +156,9 @@
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
                 v.add(unnest.getVariable());
 
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
-                        metadataProvider, policy));
+                DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid,
+                        new FeedConnectionId(dataverseName, feedName, targetDataset), metadataProvider, policy,
+                        outputType));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
@@ -194,19 +183,18 @@
         context.addPrimaryKey(pk);
     }
 
-    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
-            AqlMetadataProvider metadataProvider, FeedPolicy feedPolicy) throws AlgebricksException {
+    private AqlDataSource createFeedDataSource(AqlSourceId aqlId, FeedConnectionId feedId,
+            AqlMetadataProvider metadataProvider, FeedPolicy feedPolicy, String outputType) throws AlgebricksException {
         if (!aqlId.getDataverseName().equals(
                 metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
                         .getDataverseName())) {
             return null;
         }
-        String tName = dataset.getItemTypeName();
-        IAType itemType = metadataProvider.findType(dataset.getDataverseName(), tName);
-        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
-                AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
-        extDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
-        return extDataSource;
+        IAType feedOutputType = metadataProvider.findType(feedId.getDataverse(), outputType);
+        FeedDataSource feedDataSource = new FeedDataSource(aqlId, feedId, feedOutputType,
+                AqlDataSource.AqlDataSourceType.FEED);
+        feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
+        return feedDataSource;
     }
 
     private Pair<String, String> parseDatasetReference(AqlMetadataProvider metadataProvider, String datasetArg)
@@ -227,4 +215,23 @@
         }
         return new Pair<String, String>(dataverseName, datasetName);
     }
+
+    private String getStringArgument(AbstractFunctionCallExpression f, int index) {
+
+        ILogicalExpression expr = f.getArguments().get(index).getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return null;
+        }
+        ConstantExpression ce = (ConstantExpression) expr;
+        IAlgebricksConstantValue acv = ce.getValue();
+        if (!(acv instanceof AsterixConstantValue)) {
+            return null;
+        }
+        AsterixConstantValue acv2 = (AsterixConstantValue) acv;
+        if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
+            return null;
+        }
+        String argument = ((AString) acv2.getObject()).getStringValue();
+        return argument;
+    }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 28aee7a..334d411 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -114,7 +114,7 @@
         if (dataset == null) {
             throw new AlgebricksException("No metadata for dataset " + datasetName);
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             return false;
         }
         // Get the record type for that dataset.
@@ -129,7 +129,7 @@
     public boolean hasDataSourceScan() {
         return dataSourceScan != null;
     }
-    
+
     public void reset() {
         root = null;
         rootRef = null;
@@ -140,7 +140,7 @@
         dataset = null;
         recordType = null;
     }
-    
+
     public void getPrimaryKeyVars(List<LogicalVariable> target) {
         int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         for (int i = 0; i < numPrimaryKeys; i++) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index 20d376a..d758181 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -26,10 +26,11 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -39,6 +40,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -95,8 +97,10 @@
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
 import edu.uci.ics.asterix.metadata.declared.ResultSetDataSink;
 import edu.uci.ics.asterix.metadata.declared.ResultSetSinkId;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.functions.ExternalFunctionCompilerUtil;
@@ -153,7 +157,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -231,7 +234,7 @@
             project.getVariables().set(0, seqVar);
             resVar = seqVar;
 
-            AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+            DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                     stmt.getDatasetName());
             ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
@@ -274,7 +277,7 @@
                     leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
                     break;
                 }
-                case BEGIN_FEED: {
+                case CONNECT_FEED: {
                     ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
                             InsertDeleteOperator.Kind.INSERT);
                     insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
@@ -290,20 +293,21 @@
         return plan;
     }
 
-    private AqlDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
+    private DatasetDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
             String datasetName) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
             throw new AlgebricksException("Cannot find dataset " + datasetName + " in dataverse " + dataverseName);
         }
-
-        AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
-        String itemTypeName = dataset.getItemTypeName();
-        IAType itemType = metadataProvider.findType(dataverseName, itemTypeName);
-        AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("Cannot write output to an external dataset.");
         }
+        AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadataProvider.findType(dataverseName, itemTypeName);
+        DatasetDataSource dataSource = new DatasetDataSource(sourceId, dataset.getDataverseName(),
+                dataset.getDatasetName(), itemType, AqlDataSourceType.INTERNAL_DATASET);
+
         return dataSource;
     }
 
@@ -1275,7 +1279,7 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+    public Pair<ILogicalOperator, LogicalVariable> visitDisconnectFeedStatement(DisconnectFeedStatement del,
             Mutable<ILogicalOperator> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
@@ -1431,7 +1435,21 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedStatement(CreateFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitConnectFeedStatement(ConnectFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropFeedStatement(FeedDropStatement del,
             Mutable<ILogicalOperator> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index d1f8053..e7ef8b5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -26,10 +26,11 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -39,6 +40,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -1422,7 +1424,7 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+    public Pair<ILogicalOperator, LogicalVariable> visitDisconnectFeedStatement(DisconnectFeedStatement del,
             Mutable<ILogicalOperator> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
@@ -1443,7 +1445,21 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedStatement(CreateFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitConnectFeedStatement(ConnectFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropFeedStatement(FeedDropStatement del,
             Mutable<ILogicalOperator> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
index f2cc487..56e8ce0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
@@ -22,7 +22,6 @@
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Statement.Kind;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement.OperationType;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
@@ -39,7 +38,6 @@
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -322,16 +320,18 @@
         }
     }
 
-    public static class CompiledBeginFeedStatement implements ICompiledDmlStatement {
+    public static class CompiledConnectFeedStatement implements ICompiledDmlStatement {
         private String dataverseName;
+        private String feedName;
         private String datasetName;
         private String policyName;
         private Query query;
         private int varCounter;
 
-        public CompiledBeginFeedStatement(String dataverseName, String datasetName, String policyName, Query query,
-                int varCounter) {
+        public CompiledConnectFeedStatement(String dataverseName, String feedName, String datasetName,
+                String policyName, Query query, int varCounter) {
             this.dataverseName = dataverseName;
+            this.feedName = feedName;
             this.datasetName = datasetName;
             this.policyName = policyName;
             this.query = query;
@@ -343,6 +343,10 @@
             return dataverseName;
         }
 
+        public String getFeedName() {
+            return feedName;
+        }
+
         @Override
         public String getDatasetName() {
             return datasetName;
@@ -362,7 +366,7 @@
 
         @Override
         public Kind getKind() {
-            return Kind.BEGIN_FEED;
+            return Kind.CONNECT_FEED;
         }
 
         public String getPolicyName() {
@@ -370,20 +374,17 @@
         }
     }
 
-    public static class CompiledControlFeedStatement implements ICompiledDmlStatement {
+    public static class CompiledDisconnectFeedStatement implements ICompiledDmlStatement {
         private String dataverseName;
         private String datasetName;
-        private OperationType operationType;
+        private String feedName;
         private Query query;
         private int varCounter;
-        private Map<String, String> alteredParams;
 
-        public CompiledControlFeedStatement(OperationType operationType, String dataverseName, String datasetName,
-                Map<String, String> alteredParams) {
+        public CompiledDisconnectFeedStatement(String dataverseName, String feedName, String datasetName) {
             this.dataverseName = dataverseName;
+            this.feedName = feedName;
             this.datasetName = datasetName;
-            this.operationType = operationType;
-            this.alteredParams = alteredParams;
         }
 
         @Override
@@ -396,8 +397,8 @@
             return datasetName;
         }
 
-        public OperationType getOperationType() {
-            return operationType;
+        public String getFeedName() {
+            return feedName;
         }
 
         public int getVarCounter() {
@@ -410,16 +411,9 @@
 
         @Override
         public Kind getKind() {
-            return Kind.CONTROL_FEED;
+            return Kind.DISCONNECT_FEED;
         }
 
-        public Map<String, String> getProperties() {
-            return alteredParams;
-        }
-
-        public void setProperties(Map<String, String> properties) {
-            this.alteredParams = properties;
-        }
     }
 
     public static class CompiledDeleteStatement implements ICompiledDmlStatement {
@@ -465,8 +459,8 @@
             LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
             arguments.add(argumentLiteral);
 
-            CallExpr callExpression = new CallExpr(new FunctionSignature(
-                    FunctionConstants.ASTERIX_NS, "dataset", 1), arguments);
+            CallExpr callExpression = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, "dataset", 1),
+                    arguments);
             List<Clause> clauseList = new ArrayList<Clause>();
             Clause forClause = new ForClause(var, callExpression);
             clauseList.add(forClause);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
index 1ae4ba6..51af387 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
@@ -32,7 +32,8 @@
     protected List<Statement.Kind> getAllowedStatements() {
         Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DATAVERSE_DROP, Kind.DATASET_DECL, Kind.NODEGROUP_DECL,
                 Kind.NODEGROUP_DROP, Kind.TYPE_DECL, Kind.TYPE_DROP, Kind.CREATE_INDEX, Kind.INDEX_DECL,
-                Kind.CREATE_DATAVERSE, Kind.DATASET_DROP, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP };
+                Kind.CREATE_DATAVERSE, Kind.DATASET_DROP, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP,
+                Kind.CREATE_FEED };
         return Arrays.asList(statementsArray);
     }
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
index d83cc63..94c4df7 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
@@ -31,7 +31,7 @@
 
     protected List<Statement.Kind> getAllowedStatements() {
         Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPDATE, Kind.DML_CMD_LIST,
-                Kind.LOAD_FROM_FILE, Kind.BEGIN_FEED, Kind.CONTROL_FEED, Kind.SET };
+                Kind.LOAD_FROM_FILE, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET };
         return Arrays.asList(statementsArray);
     }
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 64e5504..66a54da 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -24,6 +24,8 @@
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.json.JSONArray;
 import org.json.JSONException;
@@ -34,18 +36,19 @@
 import edu.uci.ics.asterix.api.common.Job;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
 import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.ExternalDetailsDecl;
-import edu.uci.ics.asterix.aql.expression.FeedDetailsDecl;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FunctionDecl;
 import edu.uci.ics.asterix.aql.expression.FunctionDropStatement;
 import edu.uci.ics.asterix.aql.expression.Identifier;
@@ -84,15 +87,14 @@
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
-import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails.FeedState;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -103,11 +105,11 @@
 import edu.uci.ics.asterix.result.ResultUtils;
 import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledBeginFeedStatement;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledConnectFeedStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDeleteStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDisconnectFeedStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
@@ -134,6 +136,8 @@
  */
 public class AqlTranslator extends AbstractAqlTranslator {
 
+    private static Logger LOGGER = Logger.getLogger(AqlTranslator.class.getName());
+
     private enum ProgressState {
         NO_PROGRESS,
         ADDED_PENDINGOP_RECORD_TO_METADATA
@@ -268,13 +272,22 @@
                     break;
                 }
 
-                case BEGIN_FEED: {
-                    handleBeginFeedStatement(metadataProvider, stmt, hcc);
+                case CREATE_FEED: {
+                    handleCreateFeedStatement(metadataProvider, stmt, hcc);
                     break;
                 }
 
-                case CONTROL_FEED: {
-                    handleControlFeedStatement(metadataProvider, stmt, hcc);
+                case DROP_FEED: {
+                    handleDropFeedStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case CONNECT_FEED: {
+                    handleConnectFeedStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+
+                case DISCONNECT_FEED: {
+                    handleDisconnectFeedStatement(metadataProvider, stmt, hcc);
                     break;
                 }
 
@@ -431,28 +444,7 @@
                     datasetDetails = new ExternalDatasetDetails(adapter, properties);
                     break;
                 }
-                case FEED: {
-                    IAType itemType = dt.getDatatype();
-                    if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                        throw new AlgebricksException("Can only partition ARecord's.");
-                    }
-                    List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getPartitioningExprs();
-                    ARecordType aRecordType = (ARecordType) itemType;
-                    aRecordType.validatePartitioningExpressions(partitioningExprs);
-                    Identifier ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName();
-                    String nodegroupName = ngName != null ? ngName.getValue() : configureNodegroupForDataset(dd,
-                            dataverseName, mdTxnCtx);
-                    String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterFactoryClassname();
-                    Map<String, String> configuration = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getConfiguration();
-                    FunctionSignature signature = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getFunctionSignature();
 
-                    datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
-                            InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
-                            nodegroupName, adapter, configuration, signature);
-                    break;
-                }
             }
 
             //#. initialize DatasetIdFactory if it is not initialized.
@@ -465,7 +457,7 @@
                     DatasetIdFactory.generateDatasetId(), IMetadataEntity.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
 
-            if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
+            if (dd.getDatasetType() == DatasetType.INTERNAL) {
                 Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
                         dataverseName);
                 JobSpecification jobSpec = DatasetOperations.createDatasetJobSpec(dataverse, datasetName,
@@ -633,14 +625,17 @@
                 }
             }
 
-            if (ds.getDatasetType().equals(DatasetType.FEED)) {
-                FeedActivity fa = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx, dataverseName, datasetName,
-                        null);
-                boolean activeFeed = FeedUtil.isFeedActive(fa);
-                if (activeFeed) {
-                    throw new AsterixException("Feed " + datasetName + " is currently " + FeedState.ACTIVE + "."
-                            + " Operation not supported.");
+            List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+                    datasetName);
+            if (feedActivities != null && !feedActivities.isEmpty()) {
+                StringBuilder builder = new StringBuilder();
+
+                for (FeedActivity fa : feedActivities) {
+                    builder.append(fa + "\n");
                 }
+                throw new AsterixException("Dataset" + datasetName
+                        + " is currently being fed into by the following feeds " + "." + builder.toString()
+                        + "\nOperation not supported.");
             }
 
             //#. add a new index with PendingAddOp
@@ -799,12 +794,22 @@
                 }
             }
 
+            //# disconnect all feeds from any datasets in the dataverse.
+            List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName, null);
+            DisconnectFeedStatement disStmt = null;
+            Identifier dvId = new Identifier(dataverseName);
+            for (FeedActivity fa : feedActivities) {
+                disStmt = new DisconnectFeedStatement(dvId, new Identifier(fa.getFeedName()), new Identifier(
+                        fa.getDatasetName()));
+                handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
+            }
+
             //#. prepare jobs which will drop corresponding datasets with indexes. 
             List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
             for (int j = 0; j < datasets.size(); j++) {
                 String datasetName = datasets.get(j).getDatasetName();
                 DatasetType dsType = datasets.get(j).getDatasetType();
-                if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
+                if (dsType == DatasetType.INTERNAL) {
 
                     List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
                             datasetName);
@@ -917,17 +922,22 @@
                 }
             }
 
-            if (ds.getDatasetType().equals(DatasetType.FEED)) {
-                FeedActivity fa = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx, dataverseName, datasetName,
-                        null);
-                boolean activeFeed = FeedUtil.isFeedActive(fa);
-                if (activeFeed) {
-                    throw new AsterixException("Feed " + datasetName + " is currently " + FeedState.ACTIVE + "."
-                            + " Operation not supported.");
+            List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+                    datasetName);
+            List<JobSpecification> disconnectFeedJobSpecs = new ArrayList<JobSpecification>();
+            if (feedActivities != null && !feedActivities.isEmpty()) {
+                for (FeedActivity fa : feedActivities) {
+                    JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName,
+                            fa.getFeedName(), datasetName, metadataProvider, fa);
+                    disconnectFeedJobSpecs.add(jobSpec);
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Disconnected feed " + fa.getFeedName() + " from dataset " + datasetName
+                                + " as dataset is being dropped");
+                    }
                 }
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL) {
 
                 //#. prepare jobs to drop the datatset and the indexes in NC
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -952,6 +962,11 @@
                 bActiveTxn = false;
                 progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
 
+                //# disconnect the feeds
+                for (JobSpecification jobSpec : disconnectFeedJobSpecs) {
+                    runJob(hcc, jobSpec, true);
+                }
+
                 //#. run the jobs
                 for (JobSpecification jobSpec : jobsToExecute) {
                     runJob(hcc, jobSpec, true);
@@ -965,7 +980,7 @@
             //#. finally, delete the dataset.
             MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
             // Drop the associated nodegroup
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 String nodegroup = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
                 if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
                     MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, dataverseName + ":" + datasetName);
@@ -1035,17 +1050,20 @@
                         + dataverseName);
             }
 
-            if (ds.getDatasetType().equals(DatasetType.FEED)) {
-                FeedActivity fa = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx, dataverseName, datasetName,
-                        null);
-                boolean activeFeed = FeedUtil.isFeedActive(fa);
-                if (activeFeed) {
-                    throw new AsterixException("Feed " + datasetName + " is currently " + FeedState.ACTIVE + "."
-                            + " Operation not supported.");
+            List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+                    datasetName);
+            if (feedActivities != null && !feedActivities.isEmpty()) {
+                StringBuilder builder = new StringBuilder();
+
+                for (FeedActivity fa : feedActivities) {
+                    builder.append(fa + "\n");
                 }
+                throw new AsterixException("Dataset" + datasetName
+                        + " is currently being fed into by the following feeds " + "." + builder.toString()
+                        + "\nOperation not supported.");
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 indexName = stmtIndexDrop.getIndexName().getValue();
                 Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 if (index == null) {
@@ -1361,7 +1379,93 @@
 
     }
 
-    private void handleBeginFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+    private void handleCreateFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc) throws Exception {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dataverseName = null;
+        String feedName = null;
+        String adaptorName = null;
+        Feed feed = null;
+        try {
+            CreateFeedStatement cfs = (CreateFeedStatement) stmt;
+            dataverseName = getActiveDataverseName(cfs.getDataverseName());
+            feedName = cfs.getFeedName().getValue();
+            adaptorName = cfs.getAdaptorName();
+
+            feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
+            if (feed != null) {
+                if (cfs.getIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new AlgebricksException("A feed with this name " + adaptorName + " already exists.");
+                }
+            }
+
+            feed = new Feed(dataverseName, feedName, adaptorName, cfs.getAdaptorConfiguration(),
+                    cfs.getAppliedFunction());
+            MetadataManager.INSTANCE.addFeed(metadataProvider.getMetadataTxnContext(), feed);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        } finally {
+            releaseWriteLatch();
+        }
+    }
+
+    private void handleDropFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc) throws Exception {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
+            String dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+            String feedName = stmtFeedDrop.getFeedName().getValue();
+            Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
+            if (feed == null) {
+                if (!stmtFeedDrop.getIfExists()) {
+                    throw new AlgebricksException("There is no feed with this name " + feedName + ".");
+                }
+            }
+
+            List<FeedActivity> feedActivities;
+            try {
+                feedActivities = MetadataManager.INSTANCE.getConnectFeedActivitiesForFeed(mdTxnCtx, dataverseName,
+                        feedName);
+                MetadataManager.INSTANCE.dropFeed(mdTxnCtx, dataverseName, feedName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new MetadataException(e);
+            }
+
+            List<JobSpecification> jobSpecs = new ArrayList<JobSpecification>();
+            for (FeedActivity feedActivity : feedActivities) {
+                JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName, feedName,
+                        feedActivity.getDatasetName(), metadataProvider, feedActivity);
+                jobSpecs.add(jobSpec);
+            }
+
+            for (JobSpecification spec : jobSpecs) {
+                runJob(hcc, spec, true);
+            }
+
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        } finally {
+            releaseWriteLatch();
+        }
+    }
+
+    private void handleConnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1370,38 +1474,49 @@
         acquireReadLatch();
         boolean readLatchAcquired = true;
         try {
-            BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-            String dataverseName = getActiveDataverseName(bfs.getDataverseName());
+            ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
+            String dataverseName = getActiveDataverseName(cfs.getDataverseName());
 
-            CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName, bfs.getDatasetName()
-                    .getValue(), bfs.getPolicy(), bfs.getQuery(), bfs.getVarCounter());
+            CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(), cfs
+                    .getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
 
             Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
-                    dataverseName, bfs.getDatasetName().getValue());
+                    dataverseName, cfs.getDatasetName().getValue());
             if (dataset == null) {
-                throw new AsterixException("Unknown dataset :" + bfs.getDatasetName().getValue());
+                throw new AsterixException("Unknown target dataset :" + cfs.getDatasetName().getValue());
             }
 
-            FeedActivity recentActivity = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx, dataverseName, bfs
-                    .getDatasetName().getValue(), null);
+            if (!dataset.getDatasetType().equals(DatasetType.INTERNAL)) {
+                throw new AsterixException("Statement not applicable. Dataset " + cfs.getDatasetName().getValue()
+                        + " is not of required type " + DatasetType.INTERNAL);
+            }
+
+            Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    cfs.getFeedName());
+            if (feed == null) {
+                throw new AsterixException("Unknown source feed :" + cfs.getFeedName());
+            }
+
+            FeedActivity recentActivity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+                    new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName().getValue()), null);
             boolean isFeedActive = FeedUtil.isFeedActive(recentActivity);
-            if (isFeedActive && !bfs.isForceBegin()) {
-                throw new AsterixException("Feed " + bfs.getDatasetName().getValue()
+            if (isFeedActive && !cfs.forceConnect()) {
+                throw new AsterixException("Feed " + cfs.getDatasetName().getValue()
                         + " is currently ACTIVE. Operation not supported");
             }
             IDatasetDetails datasetDetails = dataset.getDatasetDetails();
-            if (datasetDetails.getDatasetType() != DatasetType.FEED) {
-                throw new IllegalArgumentException("Dataset " + bfs.getDatasetName().getValue()
-                        + " is not a feed dataset");
+            if (datasetDetails.getDatasetType() != DatasetType.INTERNAL) {
+                throw new IllegalArgumentException("Dataset " + cfs.getDatasetName().getValue()
+                        + " is not an interal dataset");
             }
-            bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
-            cbfs.setQuery(bfs.getQuery());
+            cfs.initialize(metadataProvider.getMetadataTxnContext(), dataset, feed);
+            cbfs.setQuery(cfs.getQuery());
             metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
             metadataProvider.getConfig().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, cbfs.getPolicyName());
-            JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, cfs.getQuery(), cbfs);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-            String waitForCompletionParam = metadataProvider.getConfig().get(BeginFeedStatement.WAIT_FOR_COMPLETION);
+            String waitForCompletionParam = metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION);
             boolean waitForCompletion = waitForCompletionParam == null ? false : Boolean
                     .valueOf(waitForCompletionParam);
             if (waitForCompletion) {
@@ -1421,7 +1536,7 @@
         }
     }
 
-    private void handleControlFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+    private void handleDisconnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
@@ -1429,7 +1544,7 @@
         acquireReadLatch();
 
         try {
-            ControlFeedStatement cfs = (ControlFeedStatement) stmt;
+            DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
             String dataverseName = getActiveDataverseName(cfs.getDataverseName());
 
             String datasetName = cfs.getDatasetName().getValue();
@@ -1439,13 +1554,19 @@
                 throw new AsterixException("Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse "
                         + dataverseName);
             }
-            if (!dataset.getDatasetType().equals(DatasetType.FEED)) {
+            if (!dataset.getDatasetType().equals(DatasetType.INTERNAL)) {
                 throw new AsterixException("Statement not applicable. Dataset " + cfs.getDatasetName().getValue()
-                        + " is not a " + DatasetType.FEED);
+                        + " is not of required type " + DatasetType.INTERNAL);
             }
 
-            FeedActivity feedActivity = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx, dataverseName,
-                    datasetName, FeedActivityType.FEED_BEGIN, FeedActivityType.FEED_RESUME);
+            Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, cfs
+                    .getFeedName().getValue());
+            if (feed == null) {
+                throw new AsterixException("Unknown source feed :" + cfs.getFeedName());
+            }
+
+            FeedActivity feedActivity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+                    new FeedConnectionId(dataverseName, feed.getFeedName(), datasetName), null);
 
             boolean isFeedActive = FeedUtil.isFeedActive(feedActivity);
             if (!isFeedActive) {
@@ -1453,16 +1574,15 @@
                         + " is currently INACTIVE. Operation not supported");
             }
 
-            CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
-                    dataverseName, cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
+            CompiledDisconnectFeedStatement clcfs = new CompiledDisconnectFeedStatement(dataverseName, cfs
+                    .getFeedName().getValue(), cfs.getDatasetName().getValue());
 
-            JobSpecification jobSpec = FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider, feedActivity);
+            JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName, cfs.getFeedName()
+                    .getValue(), cfs.getDatasetName().getValue(), metadataProvider, feedActivity);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-
             runJob(hcc, jobSpec, true);
-
         } catch (Exception e) {
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index a0872d2..7f41b4a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -200,7 +200,7 @@
         if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             throw new AsterixException("Cannot load data into dataset  (" + datasetName + ")" + "of type "
                     + dataset.getDatasetType());
         }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index 503f565..cf7ac4d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -16,18 +16,15 @@
 
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
-import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.feeds.AlterFeedMessage;
 import edu.uci.ics.asterix.metadata.feeds.FeedMessage;
 import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
 import edu.uci.ics.asterix.metadata.feeds.IFeedMessage.MessageType;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDisconnectFeedStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -56,59 +53,20 @@
      * @throws AsterixException
      * @throws AlgebricksException
      */
-    public static JobSpecification buildControlFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
-            AqlMetadataProvider metadataProvider, FeedActivity feedActivity) throws AsterixException,
-            AlgebricksException {
-        switch (controlFeedStatement.getOperationType()) {
-            case ALTER:
-            case END: {
-                return createSendMessageToFeedJobSpec(controlFeedStatement, metadataProvider, feedActivity);
-            }
-            default: {
-                throw new AsterixException("Unknown Operation Type: " + controlFeedStatement.getOperationType());
-            }
-
-        }
-    }
-
-  
-
-    private static JobSpecification createSendMessageToFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
-            AqlMetadataProvider metadataProvider, FeedActivity feedActivity) throws AsterixException {
-        String dataverseName = controlFeedStatement.getDataverseName() == null ? metadataProvider
-                .getDefaultDataverseName() : controlFeedStatement.getDataverseName();
-        String datasetName = controlFeedStatement.getDatasetName();
-        Dataset dataset;
-        try {
-            dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        } catch (AlgebricksException e) {
-            throw new AsterixException(e);
-        }
-        if (dataset == null) {
-            throw new AsterixException("FEED DATASET: No metadata for dataset " + datasetName);
-        }
-        if (dataset.getDatasetType() != DatasetType.FEED) {
-            throw new AsterixException("Operation not support for dataset type  " + dataset.getDatasetType());
-        }
+    public static JobSpecification buildDisconnectFeedJobSpec(String dataverseName, String feedName,
+            String datasetName, AqlMetadataProvider metadataProvider, FeedActivity feedActivity)
+            throws AsterixException, AlgebricksException {
 
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
         IOperatorDescriptor feedMessenger;
         AlgebricksPartitionConstraint messengerPc;
 
         IFeedMessage feedMessage = null;
-        switch (controlFeedStatement.getOperationType()) {
-            case END:
-                feedMessage = new FeedMessage(MessageType.END);
-                break;
-            case ALTER:
-                feedMessage = new AlterFeedMessage(controlFeedStatement.getProperties());
-                break;
-        }
+        feedMessage = new FeedMessage(MessageType.END);
 
         try {
             Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildFeedMessengerRuntime(
-                    metadataProvider, spec, (FeedDatasetDetails) dataset.getDatasetDetails(), dataverseName,
-                    datasetName, feedMessage, feedActivity);
+                    metadataProvider, spec, dataverseName, feedName, datasetName, feedMessage, feedActivity);
             feedMessenger = p.first;
             messengerPc = p.second;
         } catch (AlgebricksException e) {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
index 0e7ad1d..4402959 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
@@ -32,10 +32,9 @@
 import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement.OperationType;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.DataverseDecl;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.Identifier;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
@@ -54,12 +53,13 @@
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
 import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
 import edu.uci.ics.asterix.metadata.feeds.FeedPolicyAccessor;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
@@ -124,7 +124,7 @@
 
         JobSpecification spec = acggf.getJobSpecification();
         boolean feedIngestionJob = false;
-        FeedId feedId = null;
+        FeedConnectionId feedId = null;
         Map<String, String> feedPolicy = null;
         for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
             if (!(opDesc instanceof FeedIntakeOperatorDescriptor)) {
@@ -192,7 +192,8 @@
             return registeredFeeds.containsKey(jobId);
         }
 
-        public void registerFeed(FeedId feedId, JobId jobId, JobSpecification jobSpec, Map<String, String> feedPolicy) {
+        public void registerFeed(FeedConnectionId feedId, JobId jobId, JobSpecification jobSpec,
+                Map<String, String> feedPolicy) {
             if (registeredFeeds.containsKey(jobId)) {
                 throw new IllegalStateException(" Feed already registered ");
             }
@@ -209,13 +210,13 @@
                     switch (mesg.messageKind) {
                         case JOB_START:
                             if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("Job started for feed id" + feedInfo.feedId);
+                                LOGGER.info("Job started for feed id" + feedInfo.feedConnectionId);
                             }
                             handleJobStartMessage(feedInfo, mesg);
                             break;
                         case JOB_FINISH:
                             if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("Job finished for feed id" + feedInfo.feedId);
+                                LOGGER.info("Job finished for feed id" + feedInfo.feedConnectionId);
                             }
                             handleJobFinishMessage(feedInfo, mesg);
                             break;
@@ -241,14 +242,14 @@
                     AlgebricksMetaOperatorDescriptor op = ((AlgebricksMetaOperatorDescriptor) entry.getValue());
                     IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
                     for (IPushRuntimeFactory rf : runtimeFactories) {
-                        if (rf instanceof EmptyTupleSourceRuntimeFactory) {
-                            ingestOperatorIds.add(entry.getKey());
-                        } else if (rf instanceof AssignRuntimeFactory) {
+                        if (rf instanceof AssignRuntimeFactory) {
                             computeOperatorIds.add(entry.getKey());
                         }
                     }
                 } else if (entry.getValue() instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
                     storageOperatorIds.add(entry.getKey());
+                } else if (entry.getValue() instanceof FeedIntakeOperatorDescriptor) {
+                    ingestOperatorIds.add(entry.getKey());
                 }
             }
 
@@ -298,15 +299,15 @@
                 MetadataTransactionContext mdTxnCtx = null;
                 try {
                     mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                    FeedActivity fa = MetadataManager.INSTANCE.getRecentFeedActivity(mdTxnCtx,
-                            feedInfo.feedId.getDataverse(), feedInfo.feedId.getDataset(), null);
+                    FeedActivity fa = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+                            feedInfo.feedConnectionId, null);
                     FeedActivityType nextState = fa != null
                             && fa.getActivityType().equals(FeedActivityType.FEED_RECOVERY) ? FeedActivityType.FEED_RESUME
                             : FeedActivityType.FEED_BEGIN;
-                    FeedActivity feedActivity = new FeedActivity(feedInfo.feedId.getDataverse(),
-                            feedInfo.feedId.getDataset(), nextState, feedActivityDetails);
-                    MetadataManager.INSTANCE.registerFeedActivity(mdTxnCtx, new FeedId(feedInfo.feedId.getDataverse(),
-                            feedInfo.feedId.getDataset()), feedActivity);
+                    FeedActivity feedActivity = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+                            feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+                            nextState, feedActivityDetails);
+                    MetadataManager.INSTANCE.registerFeedActivity(mdTxnCtx, feedInfo.feedConnectionId, feedActivity);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 } catch (Exception e) {
                     MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
@@ -336,10 +337,13 @@
                     details.put(FeedActivity.FeedActivityDetails.EXCEPTION_MESSAGE, exceptions.get(0).getMessage());
                 }
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                FeedActivity feedActivity = new FeedActivity(feedInfo.feedId.getDataverse(),
-                        feedInfo.feedId.getDataset(), activityType, details);
-                MetadataManager.INSTANCE.registerFeedActivity(mdTxnCtx, new FeedId(feedInfo.feedId.getDataverse(),
-                        feedInfo.feedId.getDataset()), feedActivity);
+                FeedActivity feedActivity = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+                        feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+                        activityType, details);
+                MetadataManager.INSTANCE.registerFeedActivity(
+                        mdTxnCtx,
+                        new FeedConnectionId(feedInfo.feedConnectionId.getDataverse(), feedInfo.feedConnectionId
+                                .getFeedName(), feedInfo.feedConnectionId.getDatasetName()), feedActivity);
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             } catch (RemoteException | ACIDException | MetadataException e) {
                 try {
@@ -356,7 +360,7 @@
     }
 
     public static class FeedInfo {
-        public FeedId feedId;
+        public FeedConnectionId feedConnectionId;
         public JobSpecification jobSpec;
         public List<String> ingestLocations = new ArrayList<String>();
         public List<String> computeLocations = new ArrayList<String>();
@@ -364,8 +368,8 @@
         public JobInfo jobInfo;
         public Map<String, String> feedPolicy;
 
-        public FeedInfo(FeedId feedId, JobSpecification jobSpec, Map<String, String> feedPolicy) {
-            this.feedId = feedId;
+        public FeedInfo(FeedConnectionId feedId, JobSpecification jobSpec, Map<String, String> feedPolicy) {
+            this.feedConnectionId = feedId;
             this.jobSpec = jobSpec;
             this.feedPolicy = feedPolicy;
         }
@@ -375,12 +379,12 @@
             if (!(o instanceof FeedInfo)) {
                 return false;
             }
-            return ((FeedInfo) o).feedId.equals(feedId);
+            return ((FeedInfo) o).feedConnectionId.equals(feedConnectionId);
         }
 
         @Override
         public int hashCode() {
-            return feedId.hashCode();
+            return feedConnectionId.hashCode();
         }
 
     }
@@ -436,7 +440,7 @@
                 }
                 builder.append("\n");
                 for (FeedInfo fInfo : failureReport.failures.keySet()) {
-                    builder.append(fInfo.feedId);
+                    builder.append(fInfo.feedConnectionId);
                 }
                 LOGGER.warning(builder.toString());
             }
@@ -455,7 +459,7 @@
             fpa = new FeedPolicyAccessor(feedInfo.feedPolicy);
             if (!fpa.continueOnHardwareFailure()) {
                 if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Feed " + feedInfo.feedId + " is governed by policy "
+                    LOGGER.warning("Feed " + feedInfo.feedConnectionId + " is governed by policy "
                             + feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
                     LOGGER.warning("Feed policy does not require feed to recover from hardware failure. Feed will terminate");
                 }
@@ -464,7 +468,7 @@
                 // insert feed recovery mode 
                 reportFeedRecoveryMode(feedInfo);
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Feed " + feedInfo.feedId + " is governed by policy "
+                    LOGGER.info("Feed " + feedInfo.feedConnectionId + " is governed by policy "
                             + feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
                     LOGGER.info("Feed policy requires feed to recover from hardware failure. Attempting to recover feed");
                 }
@@ -492,7 +496,8 @@
                     case STORAGE_NODE:
                         recoveryPossible = false;
                         if (LOGGER.isLoggable(Level.SEVERE)) {
-                            LOGGER.severe("Unrecoverable situation! lost storage node for the feed " + feedInfo.feedId);
+                            LOGGER.severe("Unrecoverable situation! lost storage node for the feed "
+                                    + feedInfo.feedConnectionId);
                         }
                         List<String> requiredNodeIds = dependentFeeds.get(feedInfo);
                         if (requiredNodeIds == null) {
@@ -565,9 +570,10 @@
                 }
                 builder.deleteCharAt(builder.length() - 1);
                 feedActivityDetails.put(FeedActivityDetails.FEED_NODE_FAILURE, builder.toString());
-                fa = new FeedActivity(feedInfo.feedId.getDataverse(), feedInfo.feedId.getDataset(),
+                fa = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+                        feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
                         FeedActivityType.FEED_FAILURE, feedActivityDetails);
-                MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedId, fa);
+                MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedConnectionId, fa);
             }
             MetadataManager.INSTANCE.commitTransaction(ctx);
         } catch (Exception e) {
@@ -590,9 +596,9 @@
         Map<String, String> feedActivityDetails = new HashMap<String, String>();
         try {
             ctx = MetadataManager.INSTANCE.beginTransaction();
-            fa = new FeedActivity(feedInfo.feedId.getDataverse(), feedInfo.feedId.getDataset(),
-                    FeedActivityType.FEED_RECOVERY, feedActivityDetails);
-            MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedId, fa);
+            fa = new FeedActivity(feedInfo.feedConnectionId.getDataverse(), feedInfo.feedConnectionId.getFeedName(),
+                    feedInfo.feedConnectionId.getDatasetName(), FeedActivityType.FEED_RECOVERY, feedActivityDetails);
+            MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedConnectionId, fa);
 
             MetadataManager.INSTANCE.commitTransaction(ctx);
         } catch (Exception e) {
@@ -740,12 +746,12 @@
                         try {
                             JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(finfo.jobSpec);
                             if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("Resumed feed :" + finfo.feedId + " job id " + jobId);
+                                LOGGER.info("Resumed feed :" + finfo.feedConnectionId + " job id " + jobId);
                                 LOGGER.info("Job:" + finfo.jobSpec);
                             }
                         } catch (Exception e) {
                             if (LOGGER.isLoggable(Level.WARNING)) {
-                                LOGGER.warning("Unable to resume feed " + finfo.feedId + " " + e.getMessage());
+                                LOGGER.warning("Unable to resume feed " + finfo.feedConnectionId + " " + e.getMessage());
                             }
                         }
                     }
@@ -760,7 +766,7 @@
                 Thread.sleep(4000);
                 MetadataManager.INSTANCE.init();
                 ctx = MetadataManager.INSTANCE.beginTransaction();
-                List<FeedActivity> activeFeeds = MetadataManager.INSTANCE.getActiveFeeds(ctx);
+                List<FeedActivity> activeFeeds = MetadataManager.INSTANCE.getActiveFeeds(ctx, null, null);
                 if (LOGGER.isLoggable(Level.INFO)) {
                     LOGGER.info("Attempt to resume feeds that were active prior to instance shutdown!");
                     LOGGER.info("Number of feeds affected:" + activeFeeds.size());
@@ -791,7 +797,7 @@
                             LOGGER.info("Resuming feed after cluster revival: " + dataverse + ":" + datasetName
                                     + " using policy " + feedPolicy);
                         }
-                        reviveFeed(dataverse, datasetName, feedPolicy);
+                        reviveFeed(dataverse, fa.getFeedName(), datasetName, feedPolicy);
                     } else {
                         if (LOGGER.isLoggable(Level.WARNING)) {
                             LOGGER.warning("Feed " + fa.getDataverseName() + ":" + fa.getDatasetName()
@@ -815,14 +821,14 @@
             }
         }
 
-        private void reviveFeed(String dataverse, String dataset, String feedPolicy) {
+        private void reviveFeed(String dataverse, String feedName, String dataset, String feedPolicy) {
             PrintWriter writer = new PrintWriter(System.out, true);
             SessionConfig pc = new SessionConfig(true, false, false, false, false, false, true, true, false);
             try {
                 DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(dataverse));
-                BeginFeedStatement stmt = new BeginFeedStatement(new Identifier(dataverse), new Identifier(dataset),
-                        feedPolicy, 0);
-                stmt.setForceBegin(true);
+                ConnectFeedStatement stmt = new ConnectFeedStatement(new Identifier(dataverse),
+                        new Identifier(feedName), new Identifier(dataset), feedPolicy, 0);
+                stmt.setForceConnect(true);
                 List<Statement> statements = new ArrayList<Statement>();
                 statements.add(dataverseDecl);
                 statements.add(stmt);
@@ -839,7 +845,6 @@
                 }
             }
         }
-
     }
 
     public static class FeedsDeActivator implements Runnable {
@@ -863,22 +868,25 @@
             SessionConfig pc = new SessionConfig(true, false, false, false, false, false, true, true, false);
             try {
                 ctx = MetadataManager.INSTANCE.beginTransaction();
-                ControlFeedStatement stmt = new ControlFeedStatement(OperationType.END, new Identifier(
-                        feedInfo.feedId.getDataverse()), new Identifier(feedInfo.feedId.getDataset()));
+                DisconnectFeedStatement stmt = new DisconnectFeedStatement(new Identifier(
+                        feedInfo.feedConnectionId.getDataverse()), new Identifier(
+                        feedInfo.feedConnectionId.getFeedName()), new Identifier(
+                        feedInfo.feedConnectionId.getDatasetName()));
                 List<Statement> statements = new ArrayList<Statement>();
-                DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(feedInfo.feedId.getDataverse()));
+                DataverseDecl dataverseDecl = new DataverseDecl(
+                        new Identifier(feedInfo.feedConnectionId.getDataverse()));
                 statements.add(dataverseDecl);
                 statements.add(stmt);
                 AqlTranslator translator = new AqlTranslator(statements, writer, pc, DisplayFormat.TEXT);
                 translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, false);
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("End urecoverable feed: " + feedInfo.feedId.getDataverse() + ":"
-                            + feedInfo.feedId.getDataset());
+                    LOGGER.info("End urecoverable feed: " + feedInfo.feedConnectionId);
                 }
                 MetadataManager.INSTANCE.commitTransaction(ctx);
             } catch (Exception e) {
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Exception in ending loser feed: " + feedInfo.feedId + " Exception " + e.getMessage());
+                    LOGGER.info("Exception in ending loser feed: " + feedInfo.feedConnectionId + " Exception "
+                            + e.getMessage());
                 }
                 e.printStackTrace();
                 try {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
index f7622be..e9039bc 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
@@ -178,7 +178,7 @@
         if (replacementNode == null) {
             if (LOGGER.isLoggable(Level.SEVERE)) {
                 LOGGER.severe("Unable to find replacement for failed node :" + failedNodeId);
-                LOGGER.severe("Feed: " + feedInfo.feedId + " will be terminated");
+                LOGGER.severe("Feed: " + feedInfo.feedConnectionId + " will be terminated");
             }
             List<FeedInfo> feedsToTerminate = new ArrayList<FeedInfo>();
             feedsToTerminate.add(feedInfo);
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
index 91b75c5..6afc331 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
@@ -105,7 +105,9 @@
 
     @Test
     public void test() throws Exception {
+        // if (tcCtx.getTestCase().getCompilationUnit().get(0).getName().contains("meta09")) {
         TestsUtils.executeTest(PATH_ACTUAL, tcCtx);
+        // }
     }
 
 }
\ No newline at end of file
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index c4672ca..21caf28 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -85,10 +85,6 @@
         System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
 
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Set up complete");
-        }
-
     }
 
     @AfterClass
@@ -119,7 +115,6 @@
 
     @Parameters
     public static Collection<Object[]> tests() throws Exception {
-        System.out.println("starting collection");
         Collection<Object[]> testArgs = new ArrayList<Object[]>();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         for (TestCaseContext ctx : b.build(new File(PATH_BASE))) {
@@ -134,13 +129,8 @@
         this.tcCtx = tcCtx;
     }
 
-    // issue_251_dataset_hint
-    //  feed_01,02,03,04,issue_230_feeds  
-    // 
     @Test
     public void test() throws Exception {
-        //    if (tcCtx.getTestCase().getCompilationUnit().get(0).getName().contains("issue_230_feeds")) {
         TestsUtils.executeTest(PATH_ACTUAL, tcCtx);
-        //  }
     }
 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
index 7c82b18..7a6c4e0 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:11:26 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 23:01:41 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
index 7c82b18..7a6c4e0 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:11:26 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 23:01:41 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
index f931b40..991fa32 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:00:38 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 22:49:42 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
index f931b40..991fa32 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:00:38 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 22:49:42 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
index efd3a7e..7aca625 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:59:57 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:00:46 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
index efd3a7e..7aca625 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:59:57 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:00:46 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta02.adm b/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
index 394af5c..027f12b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:34 PST 2013" }
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:35:01 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
index 394af5c..027f12b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:34 PST 2013" }
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:35:01 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta09.adm b/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
index 9bcb2a4..d0ec344 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:55:25 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:28:17 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
index 9bcb2a4..d0ec344 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:55:25 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:28:17 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta15.adm b/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
index b55d936..43cf9ec 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
@@ -1,9 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "file_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "synthetic_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
index b55d936..43cf9ec 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
@@ -1,9 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "file_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "synthetic_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
index 89a1dc9..aecf8a8 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index 89a1dc9..aecf8a8 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
index 03b73f7..686aea5 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
@@ -1,69 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "UpdateTimestamp", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index 03b73f7..686aea5 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,69 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "UpdateTimestamp", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
index 00578f9..ee088a1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
@@ -1,14 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
index 00578f9..ee088a1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
@@ -1,14 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:30:03 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
index c4e0eaa..2de7431 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index c4e0eaa..2de7431 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
index e7be7c6..ea5e89b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
@@ -1,69 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "UpdateTimestamp", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index e7be7c6..ea5e89b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,69 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "UpdateTimestamp", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
index 51aa08e..a4ee9d3 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
@@ -1,14 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index 51aa08e..a4ee9d3 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,14 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:10 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Jul 11 18:40:11 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
index 4705cce..ef2df84 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
@@ -15,8 +15,9 @@
   timestamp : string
 }      
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
 primary key id;
 
+create feed TweetFeed
+using file_feed
+(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
index 5ee2e87..392e471 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
@@ -4,6 +4,6 @@
  * Date         : 24th Dec 2012
  */
 
-for $x in dataset('Metadata.Dataset')
-where $x.DataverseName='feeds' and $x.DatasetName='TweetFeed'
+for $x in dataset('Metadata.Feed')
+where $x.DataverseName='feeds' and $x.FeedName='TweetFeed'
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
index 1f6b30d..fe5b88f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
@@ -16,11 +16,11 @@
   location : string,
   text : string,
   timestamp : string
-}      
+};      
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
 primary key id;
 
-
+create feed  TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
index 26b1469..3f06b7a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
@@ -9,4 +9,4 @@
 
 set wait-for-completion-feed "true";
 
-begin feed TweetFeed;
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
index 230aa40..3230002 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
@@ -7,6 +7,6 @@
 
 use dataverse feeds;
 
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
 order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
index 488ed2f..8cdeb9b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
@@ -16,13 +16,15 @@
   timestamp : string
 }      
 
+create dataset Tweets(TweetType)
+primary key id;
+
 create function feed_processor($x) {
 $x
 }
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
+create feed TweetFeed
+using file_feed
 (("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
-apply function feed_processor@1 
-primary key id;
+apply function feed_processor@1; 
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
index 1922f39..a7ee344 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
@@ -4,6 +4,6 @@
  * Date         : 24th Dec 2012
  */
 
-for $x in dataset('Metadata.Dataset')
-where $x.DataverseName='feeds' and $x.DatasetName='TweetFeed'
+for $x in dataset('Metadata.Feed')
+where $x.DataverseName='feeds' and $x.FeedName='TweetFeed'
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
index 326b2d5..6f539b0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
@@ -17,8 +17,9 @@
   timestamp : string
 }      
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="hdfs"),("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/obamatweets.adm"),("format"="adm"),("input-format"="text-input-format"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
 primary key id;
 
+create feed TweetFeed
+using file_feed
+(("fs"="hdfs"),("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/obamatweets.adm"),("format"="adm"),("input-format"="text-input-format"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
index c0d70b9..e87111c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
@@ -10,4 +10,4 @@
 
 set wait-for-completion-feed "true";
 
-begin feed TweetFeed;
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
index 714dd80..451acde 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
@@ -7,6 +7,6 @@
  */
 use dataverse feeds;
 
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
 order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
index bdeed63..ec92972 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
@@ -29,9 +29,11 @@
 	message-text: string
 }
 
-
-create feed dataset SyntheticTweetFeed(TweetMessageType)
-using "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory"
-(("duration"="5"),("tps"="2"),("dataverse-dataset"="feeds:SyntheticTweetFeed"))
+create dataset SyntheticTweets(TweetMessageType)
 primary key tweetid;
 
+
+create feed  SyntheticTweetFeed
+using synthetic_twitter_feed
+(("duration"="5"),("tps"="2"),("dataverse-dataset"="feeds:SyntheticTweets"))
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
index 378cada..d9e5404 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
@@ -12,4 +12,4 @@
 
 set wait-for-completion-feed "true";
 
-begin feed SyntheticTweetFeed;
+connect feed SyntheticTweetFeed to dataset SyntheticTweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
index a9ce3a4..dd520da 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
@@ -1,6 +1,7 @@
 /*
  * Description  : Create a feed dataset that uses the synthetic feed simulator adapter.
-                  The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed. The                   feed lasts a configured duration with data arriving at a configured rate (tweets per second).
+                  The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed. 
+                  The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
                   Verify the existence of data after the feed finishes.
 
  * Expected Res : Success
@@ -10,7 +11,7 @@
 use dataverse feeds;
 
 let $totalTweets:=count(
-for $x in dataset('SyntheticTweetFeed')
+for $x in dataset('SyntheticTweets')
 return $x)
 return 
 (if($totalTweets > 0)
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
index f61f320..805a41c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
@@ -17,8 +17,11 @@
   timestamp : string
 }      
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
 primary key id;
 
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
index 4919a34..fc71769 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
@@ -9,4 +9,4 @@
 
 set wait-for-completion-feed "true";
  
-begin feed feeds.TweetFeed;
+connect feed feeds.TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
index 48e18e2..227913d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
@@ -6,6 +6,6 @@
  */
 use dataverse feeds;
 
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
 order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
index ca09806..92f716d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
@@ -17,9 +17,10 @@
   timestamp : string
 }      
 
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
 primary key id
 hints(cardinality=200);
 
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
index 7c08919..19e0930d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
@@ -10,4 +10,4 @@
 
 set wait-for-completion-feed "true";
 
-begin feed feeds.TweetFeed;
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
index f1127f0..7ee0dcf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
@@ -8,7 +8,7 @@
 
 use dataverse feeds;
 
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
 order by $x.id
 return $x
 
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
index 8bd73db..7b444ed 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
@@ -1,4 +1,4 @@
-{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:01:46 PST 2013" }
-{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:01:46 PST 2013" }
-{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:01:46 PST 2013" }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:01:46 PST 2013" }
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:06 PDT 2013", "DatasetId": 656, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:06 PDT 2013", "DatasetId": 655, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 657, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 658, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
index ff1f9df..29c93d4 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
@@ -1,4 +1,4 @@
-{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:03:50 PST 2013" }
-{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:03:50 PST 2013" }
-{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:03:50 PST 2013" }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:03:50 PST 2013" }
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 668, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 667, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 669, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 670, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 7be9c57..db83554 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 678, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 672, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 675, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 673, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 674, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 677, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
index fb4e84e..b425120 100644
--- a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": null }, "Hints": {{  }}, "Timestamp": "Tue Jun 11 13:56:43 PDT 2013", "DatasetId": 696, "PendingOp": 0 }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdaptorName": "file_feed", "AdaptorConfiguration": {{ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } }}, "Function": null, "Timestamp": "Wed Jul 17 08:26:24 PDT 2013" }
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
index d8a0c3f..2ea09d7 100644
--- a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": "feeds.feed_processor@1" }, "Hints": {{  }}, "Timestamp": "Tue Jun 11 12:28:32 PDT 2013", "DatasetId": 698, "PendingOp": 0 }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdaptorName": "file_feed", "AdaptorConfiguration": {{ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } }}, "Function": "feeds.feed_processor@1", "Timestamp": "Wed Jul 17 08:06:52 PDT 2013" }
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
index c42d848..a1b67d4 100644
--- a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
@@ -1,6 +1,6 @@
-{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:32:02 PDT 2013", "DatasetId": 366, "PendingOp": 0 }
-{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:32:02 PDT 2013", "DatasetId": 367, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:56 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:57 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:57 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:56 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:45 PDT 2013", "DatasetId": 369, "PendingOp": 0 }
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:45 PDT 2013", "DatasetId": 370, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
index 98f8387..eede67b 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
@@ -37,8 +37,10 @@
         INDEX_DECL,
         CREATE_DATAVERSE,
         INDEX_DROP,
-        BEGIN_FEED,
-        CONTROL_FEED,
+        CREATE_FEED,
+        DROP_FEED,
+        CONNECT_FEED,
+        DISCONNECT_FEED,
         CREATE_FUNCTION,
         FUNCTION_DROP
     }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
deleted file mode 100644
index 8262b41..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.aql.expression;
-
-import java.io.StringReader;
-import java.util.List;
-
-import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
-import edu.uci.ics.asterix.aql.parser.AQLParser;
-import edu.uci.ics.asterix.aql.parser.ParseException;
-import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
-import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.Function;
-import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
-
-public class BeginFeedStatement implements Statement {
-
-    private final Identifier dataverseName;
-    private final Identifier datasetName;
-    private final String policy;
-    private Query query;
-    private int varCounter;
-    private boolean forceBegin = false;
-
-    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
-
-    public BeginFeedStatement(Identifier dataverseName, Identifier datasetName, String policy, int varCounter) {
-        this.dataverseName = dataverseName;
-        this.datasetName = datasetName;
-        this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
-        this.varCounter = varCounter;
-    }
-
-    public void initialize(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException {
-        query = new Query();
-        FeedDatasetDetails feedDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
-        String functionName = feedDetails.getFunction() == null ? null : feedDetails.getFunction().getName();
-        StringBuilder builder = new StringBuilder();
-        builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
-        builder.append("insert into dataset " + datasetName + " ");
-
-        if (functionName == null) {
-            builder.append(" (" + " for $x in feed-ingest ('" + datasetName + "') ");
-            builder.append(" return $x");
-        } else {
-            int arity = feedDetails.getFunction().getArity();
-            FunctionSignature signature = new FunctionSignature(dataset.getDataverseName(), functionName, arity);
-            Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
-            if (function == null) {
-                throw new MetadataException(" Unknown function " + feedDetails.getFunction());
-            }
-            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
-                String param = function.getParams().get(0);
-                builder.append(" (" + " for" + " " + param + " in feed-ingest ('" + datasetName + "') ");
-                builder.append(" let $y:=(" + function.getFunctionBody() + ")" + " return $y");
-            } else {
-                builder.append(" (" + " for $x in feed-ingest ('" + datasetName + "') ");
-                builder.append(" let $y:=" + dataset.getDataverseName() + "." + function.getName() + "(" + "$x" + ")");
-                builder.append(" return $y");
-            }
-
-        }
-        builder.append(")");
-        builder.append(";");
-        AQLParser parser = new AQLParser(new StringReader(builder.toString()));
-
-        List<Statement> statements;
-        try {
-            statements = parser.Statement();
-            query = ((InsertStatement) statements.get(1)).getQuery();
-        } catch (ParseException pe) {
-            throw new MetadataException(pe);
-        }
-
-    }
-
-    public Identifier getDataverseName() {
-        return dataverseName;
-    }
-
-    public Identifier getDatasetName() {
-        return datasetName;
-    }
-
-    public Query getQuery() {
-        return query;
-    }
-
-    public int getVarCounter() {
-        return varCounter;
-    }
-
-    @Override
-    public Kind getKind() {
-        return Kind.BEGIN_FEED;
-    }
-
-    public String getPolicy() {
-        return policy;
-    }
-
-    @Override
-    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
-        return visitor.visitBeginFeedStatement(this, arg);
-    }
-
-    @Override
-    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
-        visitor.visit(this, arg);
-    }
-
-    public boolean isForceBegin() {
-        return forceBegin;
-    }
-
-    public void setForceBegin(boolean forceBegin) {
-        this.forceBegin = forceBegin;
-    }
-
-}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java
new file mode 100644
index 0000000..3ee53a6
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java
@@ -0,0 +1,180 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import java.io.StringReader;
+import java.util.List;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.aql.parser.AQLParser;
+import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+public class ConnectFeedStatement implements Statement {
+
+    private final Identifier dataverseName;
+    private final Identifier datasetName;
+    private final String feedName;
+    private final String policy;
+    private Query query;
+    private int varCounter;
+    private boolean forceConnect = false;
+
+    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+
+    public ConnectFeedStatement(Pair<Identifier, Identifier> feedNameCmp, Pair<Identifier, Identifier> datasetNameCmp,
+            String policy, int varCounter) {
+        if (feedNameCmp.first != null && datasetNameCmp.first != null
+                && !feedNameCmp.first.getValue().equals(datasetNameCmp.first.getValue())) {
+            throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
+        }
+        this.dataverseName = feedNameCmp.first != null ? feedNameCmp.first
+                : datasetNameCmp.first != null ? datasetNameCmp.first : null;
+        this.datasetName = datasetNameCmp.second;
+        this.feedName = feedNameCmp.second.getValue();
+        this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
+        this.varCounter = varCounter;
+    }
+
+    public ConnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName, String policy,
+            int varCounter) {
+        this.dataverseName = dataverseName;
+        this.datasetName = datasetName;
+        this.feedName = feedName.getValue();
+        this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
+        this.varCounter = varCounter;
+    }
+
+    public void initialize(MetadataTransactionContext mdTxnCtx, Dataset targetDataset, Feed sourceFeed)
+            throws MetadataException {
+        query = new Query();
+        FunctionSignature appliedFunction = sourceFeed.getAppliedFunction();
+        Function function = null;
+        String adaptorOutputType = null;
+        if (appliedFunction != null) {
+            function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
+            if (function == null) {
+                throw new MetadataException(" Unknown function " + function);
+            } else {
+                if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+                    adaptorOutputType = targetDataset.getItemTypeName();
+                } else {
+                    if (function.getParams().size() > 1) {
+                        throw new MetadataException(" Incompatible function: " + appliedFunction
+                                + " Number if arguments must be 1");
+                    }
+                    adaptorOutputType = function.getParams().get(0);
+                }
+            }
+        } else {
+            adaptorOutputType = targetDataset.getItemTypeName();
+        }
+        StringBuilder builder = new StringBuilder();
+        builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
+        builder.append("insert into dataset " + datasetName + " ");
+
+        if (appliedFunction == null) {
+            builder.append(" (" + " for $x in feed-ingest ('" + feedName + "'" + "," + "'" + adaptorOutputType + "'"
+                    + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+            builder.append(" return $x");
+        } else {
+            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+                String param = function.getParams().get(0);
+                builder.append(" (" + " for" + " " + param + " in feed-ingest ('" + feedName + "'" + "," + "'"
+                        + adaptorOutputType + "'" + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+                builder.append(" let $y:=(" + function.getFunctionBody() + ")" + " return $y");
+            } else {
+                builder.append(" (" + " for $x in feed-ingest ('" + feedName + "'" + "," + "'" + adaptorOutputType
+                        + "'" + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+                builder.append(" let $y:=" + sourceFeed.getDataverseName() + "." + function.getName() + "(" + "$x"
+                        + ")");
+                builder.append(" return $y");
+            }
+
+        }
+        builder.append(")");
+        builder.append(";");
+        AQLParser parser = new AQLParser(new StringReader(builder.toString()));
+
+        List<Statement> statements;
+        try {
+            statements = parser.Statement();
+            query = ((InsertStatement) statements.get(1)).getQuery();
+        } catch (ParseException pe) {
+            throw new MetadataException(pe);
+        }
+
+    }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getDatasetName() {
+        return datasetName;
+    }
+
+    public Query getQuery() {
+        return query;
+    }
+
+    public int getVarCounter() {
+        return varCounter;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CONNECT_FEED;
+    }
+
+    public String getPolicy() {
+        return policy;
+    }
+
+    @Override
+    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+        return visitor.visitConnectFeedStatement(this, arg);
+    }
+
+    @Override
+    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+        visitor.visit(this, arg);
+    }
+
+    public boolean forceConnect() {
+        return forceConnect;
+    }
+
+    public void setForceConnect(boolean forceConnect) {
+        this.forceConnect = forceConnect;
+    }
+
+    public String getFeedName() {
+        return feedName;
+    }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java
deleted file mode 100644
index d442fe7..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.aql.expression;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-public class ControlFeedStatement implements Statement {
-
-    private final Identifier dataverseName;
-    private final Identifier datasetName;
-
-    public enum OperationType {
-        BEGIN,
-        SUSPEND,
-        RESUME,
-        END,
-        ALTER
-    }
-
-    private OperationType operationType;
-    private Map<String, String> alterAdapterConfParams;
-
-    public ControlFeedStatement(OperationType operation, Identifier dataverseName, Identifier datasetName) {
-        this.operationType = operation;
-        this.datasetName = datasetName;
-        this.dataverseName = dataverseName;
-    }
-
-    public ControlFeedStatement(OperationType operation, Identifier dataverseName, Identifier datasetName,
-            Map<String, String> alterAdapterConfParams) {
-        this.operationType = operation;
-        this.datasetName = datasetName;
-        this.dataverseName = dataverseName;
-        this.alterAdapterConfParams = alterAdapterConfParams;
-    }
-
-    public Identifier getDataverseName() {
-        return dataverseName;
-    }
-
-    public Identifier getDatasetName() {
-        return datasetName;
-    }
-
-    public OperationType getOperationType() {
-        return operationType;
-    }
-
-    public void setOperation(OperationType operationType) {
-        this.operationType = operationType;
-    }
-
-    @Override
-    public Kind getKind() {
-        return Kind.CONTROL_FEED;
-    }
-
-    public Map<String, String> getAlterAdapterConfParams() {
-        return alterAdapterConfParams;
-    }
-
-    @Override
-    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
-        return visitor.visitControlFeedStatement(this, arg);
-    }
-
-    @Override
-    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
-        visitor.visit(this, arg);
-    }
-
-}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java
new file mode 100644
index 0000000..8f90e87
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+
+public class CreateFeedStatement implements Statement {
+
+    private final Identifier dataverseName;
+    private final Identifier feedName;
+    private final String adaptorName;
+    private final Map<String, String> adaptorConfiguration;
+    private final FunctionSignature appliedFunction;
+    private final boolean ifNotExists;
+
+    public CreateFeedStatement(Identifier dataverseName, Identifier feedName, String adaptorName,
+            Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
+        this.feedName = feedName;
+        this.dataverseName = dataverseName;
+        this.adaptorName = adaptorName;
+        this.adaptorConfiguration = adaptorConfiguration;
+        this.appliedFunction = appliedFunction;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getFeedName() {
+        return feedName;
+    }
+
+    public String getAdaptorName() {
+        return adaptorName;
+    }
+
+    public Map<String, String> getAdaptorConfiguration() {
+        return adaptorConfiguration;
+    }
+
+    public FunctionSignature getAppliedFunction() {
+        return appliedFunction;
+    }
+
+    public boolean getIfNotExists() {
+        return this.ifNotExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CREATE_FEED;
+    }
+
+    @Override
+    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+        return visitor.visitCreateFeedStatement(this, arg);
+    }
+
+    @Override
+    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+        visitor.visit(this, arg);
+    }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java
new file mode 100644
index 0000000..b2035a6
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+public class DisconnectFeedStatement implements Statement {
+
+    private final Identifier dataverseName;
+    private final Identifier feedName;
+    private final Identifier datasetName;
+
+    public DisconnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName) {
+        this.feedName = feedName;
+        this.datasetName = datasetName;
+        this.dataverseName = dataverseName;
+    }
+
+    public DisconnectFeedStatement(Pair<Identifier, Identifier> feedNameComponent,
+            Pair<Identifier, Identifier> datasetNameComponent) {
+        if (feedNameComponent.first != null && datasetNameComponent.first != null
+                && !feedNameComponent.first.getValue().equals(datasetNameComponent.first.getValue())) {
+            throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
+        }
+        this.dataverseName = feedNameComponent.first != null ? feedNameComponent.first
+                : datasetNameComponent.first != null ? datasetNameComponent.first : null;
+        this.datasetName = datasetNameComponent.second;
+        this.feedName = feedNameComponent.second;
+    }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getFeedName() {
+        return feedName;
+    }
+
+    public Identifier getDatasetName() {
+        return datasetName;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.DISCONNECT_FEED;
+    }
+
+    @Override
+    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+        return visitor.visitDisconnectFeedStatement(this, arg);
+    }
+
+    @Override
+    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+        visitor.visit(this, arg);
+    }
+
+    @Override
+    public String toString() {
+        return "disconnect feed " + feedName + " from " + datasetName;
+    }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java
new file mode 100644
index 0000000..0120a7d
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class FeedDropStatement implements Statement {
+
+    private final Identifier dataverseName;
+    private final Identifier feedName;
+    private boolean ifExists;
+
+    public FeedDropStatement(Identifier dataverseName, Identifier feedName, boolean ifExists) {
+        this.dataverseName = dataverseName;
+        this.feedName = feedName;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.DROP_FEED;
+    }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getFeedName() {
+        return feedName;
+    }
+
+    public boolean getIfExists() {
+        return ifExists;
+    }
+
+    @Override
+    public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+        return visitor.visitDropFeedStatement(this, arg);
+    }
+
+    @Override
+    public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+        visitor.visit(this, arg);
+    }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
index cdbd86f..3a2c897 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
@@ -21,10 +21,11 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Literal;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -34,6 +35,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -397,9 +399,6 @@
         } else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
             out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
                     + "is an external dataset");
-        } else if (dd.getDatasetType() == DatasetType.FEED) {
-            out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
-                    + "is an feed dataset");
         }
     }
 
@@ -446,8 +445,8 @@
     }
 
     @Override
-    public void visit(ControlFeedStatement ss, Integer step) throws AsterixException {
-        out.println(skip(step) + ss.getOperationType() + skip(step) + ss.getDatasetName());
+    public void visit(DisconnectFeedStatement ss, Integer step) throws AsterixException {
+        out.println(skip(step) + skip(step) + ss.getFeedName() + skip(step) + ss.getDatasetName());
     }
 
     @Override
@@ -528,7 +527,19 @@
     }
 
     @Override
-    public void visit(BeginFeedStatement stmtDel, Integer arg) throws AsterixException {
+    public void visit(CreateFeedStatement stmtDel, Integer arg) throws AsterixException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void visit(ConnectFeedStatement stmtDel, Integer arg) throws AsterixException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void visit(FeedDropStatement stmt, Integer arg) throws AsterixException {
         // TODO Auto-generated method stub
 
     }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
index 7fbf342..39f4d90 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
@@ -14,17 +14,19 @@
  */
 package edu.uci.ics.asterix.aql.expression.visitor;
 
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
 import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
@@ -150,9 +152,13 @@
 
     R visitSetStatement(SetStatement ss, T arg) throws AsterixException;
 
-    R visitBeginFeedStatement(BeginFeedStatement bf, T arg) throws AsterixException;
+    R visitDisconnectFeedStatement(DisconnectFeedStatement del, T arg) throws AsterixException;
 
-    R visitControlFeedStatement(ControlFeedStatement del, T arg) throws AsterixException;
+    R visitConnectFeedStatement(ConnectFeedStatement del, T arg) throws AsterixException;
+
+    R visitCreateFeedStatement(CreateFeedStatement del, T arg) throws AsterixException;
+
+    R visitDropFeedStatement(FeedDropStatement del, T arg) throws AsterixException;
 
     R visitCallExpr(CallExpr pf, T arg) throws AsterixException;
 
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
index b936c63..d47c3ab 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
@@ -14,17 +14,19 @@
  */
 package edu.uci.ics.asterix.aql.expression.visitor;
 
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDecl;
 import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
 import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
@@ -154,9 +156,13 @@
 
     void visit(TypeDropStatement stmtDel, T arg) throws AsterixException;
 
-    void visit(BeginFeedStatement stmtDel, T arg) throws AsterixException;
+    void visit(DisconnectFeedStatement stmtDel, T arg) throws AsterixException;
 
-    void visit(ControlFeedStatement stmtDel, T arg) throws AsterixException;
+    void visit(ConnectFeedStatement stmtDel, T arg) throws AsterixException;
+
+    void visit(CreateFeedStatement stmt, T arg) throws AsterixException;
+
+    void visit(FeedDropStatement stmt, T arg) throws AsterixException;
 
     void visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
 
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
index b05b308..36d045d 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
@@ -24,10 +24,11 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -37,6 +38,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -532,7 +534,7 @@
         }
 
         @Override
-        public Void visitControlFeedStatement(ControlFeedStatement del, Void arg) throws AsterixException {
+        public Void visitDisconnectFeedStatement(DisconnectFeedStatement del, Void arg) throws AsterixException {
             // TODO Auto-generated method stub
             return null;
         }
@@ -550,7 +552,19 @@
         }
 
         @Override
-        public Void visitBeginFeedStatement(BeginFeedStatement bf, Void arg) throws AsterixException {
+        public Void visitCreateFeedStatement(CreateFeedStatement del, Void arg) throws AsterixException {
+            // TODO Auto-generated method stub
+            return null;
+        }
+
+        @Override
+        public Void visitConnectFeedStatement(ConnectFeedStatement del, Void arg) throws AsterixException {
+            // TODO Auto-generated method stub
+            return null;
+        }
+
+        @Override
+        public Void visitDropFeedStatement(FeedDropStatement del, Void arg) throws AsterixException {
             // TODO Auto-generated method stub
             return null;
         }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
index dcb4a66..b3edca2 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
@@ -21,10 +21,11 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.IAqlExpression;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -34,6 +35,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -409,7 +411,7 @@
     }
 
     @Override
-    public Pair<IAqlExpression, List<VariableSubstitution>> visitControlFeedStatement(ControlFeedStatement del,
+    public Pair<IAqlExpression, List<VariableSubstitution>> visitDisconnectFeedStatement(DisconnectFeedStatement del,
             List<VariableSubstitution> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
@@ -577,7 +579,21 @@
     }
 
     @Override
-    public Pair<IAqlExpression, List<VariableSubstitution>> visitBeginFeedStatement(BeginFeedStatement bf,
+    public Pair<IAqlExpression, List<VariableSubstitution>> visitCreateFeedStatement(CreateFeedStatement del,
+            List<VariableSubstitution> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<IAqlExpression, List<VariableSubstitution>> visitConnectFeedStatement(ConnectFeedStatement del,
+            List<VariableSubstitution> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<IAqlExpression, List<VariableSubstitution>> visitDropFeedStatement(FeedDropStatement del,
             List<VariableSubstitution> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
index ae5d7d9..d9a7136 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
@@ -22,10 +22,11 @@
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Expression.Kind;
 import edu.uci.ics.asterix.aql.base.IAqlExpression;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
 import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
 import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -35,6 +36,7 @@
 import edu.uci.ics.asterix.aql.expression.DistinctClause;
 import edu.uci.ics.asterix.aql.expression.DropStatement;
 import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
 import edu.uci.ics.asterix.aql.expression.FieldAccessor;
 import edu.uci.ics.asterix.aql.expression.FieldBinding;
 import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -499,7 +501,8 @@
     }
 
     @Override
-    public Boolean visitControlFeedStatement(ControlFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+    public Boolean visitDisconnectFeedStatement(DisconnectFeedStatement del, List<FunctionDecl> arg)
+            throws AsterixException {
         // TODO Auto-generated method stub
         return null;
     }
@@ -518,7 +521,19 @@
     }
 
     @Override
-    public Boolean visitBeginFeedStatement(BeginFeedStatement bf, List<FunctionDecl> arg) throws AsterixException {
+    public Boolean visitCreateFeedStatement(CreateFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Boolean visitConnectFeedStatement(ConnectFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Boolean visitDropFeedStatement(FeedDropStatement del, List<FunctionDecl> arg) throws AsterixException {
         // TODO Auto-generated method stub
         return null;
     }
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index c62edcf..d71db17 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -195,6 +195,7 @@
     | stmt = IndexSpecification()
     | stmt = DataverseSpecification()
     | stmt = FunctionSpecification()
+    | stmt = FeedSpecification()
   )        
   {
     return stmt;
@@ -284,29 +285,6 @@
                                    ifNotExists);
       } 
 
-    | "feed" <DATASET> nameComponents = QualifiedName()
-    <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
-    ifNotExists = IfNotExists()
-    "using" adapterName = AdapterName() properties = Configuration()
-    (appliedFunction = ApplyFunction())? primaryKeyFields = PrimaryKey()
-    ( "on" nodeGroupName = Identifier() )?
-    ( "hints" hints = Properties() )?
-      {
-        FeedDetailsDecl fdd = new FeedDetailsDecl(adapterName,
-                                                  properties,
-                                                  appliedFunction,
-                                                  nodeGroupName != null
-                                                    ? new Identifier(nodeGroupName)
-                                                    : null,
-                                                  primaryKeyFields);
-        dsetDecl = new DatasetDecl(nameComponents.first,
-                                   nameComponents.second,
-                                   new Identifier(typeName),
-                                   hints,
-                                   DatasetType.FEED,
-                                   fdd,
-                                   ifNotExists);
-      }
     | ("internal")? <DATASET> nameComponents = QualifiedName()
     <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
     ifNotExists = IfNotExists()
@@ -456,6 +434,34 @@
     }
 }
 
+CreateFeedStatement FeedSpecification() throws ParseException:
+{
+  Pair<Identifier,Identifier> nameComponents = null;  
+  boolean ifNotExists = false;
+  String adaptorName = null;
+  Map<String,String> properties = null;
+  FunctionSignature appliedFunction = null;
+  CreateFeedStatement cfs = null;
+}
+{
+  (
+    "feed"  nameComponents = QualifiedName()
+    ifNotExists = IfNotExists()
+    "using" adaptorName = AdapterName() properties = Configuration()
+    (appliedFunction = ApplyFunction())?
+      {
+        cfs = new CreateFeedStatement(nameComponents.first,
+                                   nameComponents.second, adaptorName, properties, appliedFunction, ifNotExists);
+      } 
+      
+  )
+    {
+      return cfs;
+    }
+}
+
+
+
 List<VarIdentifier> ParameterList() throws ParseException:
 {
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
@@ -594,6 +600,10 @@
       {
         stmt = new FunctionDropStatement(funcSig, ifExists);
       }
+    | "feed" pairId = QualifiedName() ifExists = IfExists()
+      {
+        stmt = new FeedDropStatement(pairId.first, pairId.second, ifExists);
+      }  
   )
   {
     return stmt;  
@@ -761,32 +771,22 @@
 
 Statement FeedStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<Identifier,Identifier> feedNameComponents = null;
+  Pair<Identifier,Identifier> datasetNameComponents = null;
+  
   Map<String,String> configuration = null;
   Statement stmt = null;
   String policy = null;
 }
 {
   (
-    "begin" "feed" nameComponents = QualifiedName()  (policy = GetPolicy())? 
+    "connect" "feed" feedNameComponents = QualifiedName() "to" <DATASET> datasetNameComponents = QualifiedName() (policy = GetPolicy())? 
       {
-        stmt = new BeginFeedStatement(nameComponents.first, nameComponents.second, policy, getVarCounter());
+        stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
       }
-    | "suspend" "feed" nameComponents = QualifiedName()
+    | "disconnect" "feed" feedNameComponents = QualifiedName() "from" <DATASET> datasetNameComponents = QualifiedName()
       {
-        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.SUSPEND, nameComponents.first, nameComponents.second);
-      }
-    | "resume" "feed" nameComponents = QualifiedName()
-      {
-        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.RESUME, nameComponents.first, nameComponents.second);
-      }
-    | "end" "feed" nameComponents = QualifiedName()
-      {
-        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.END, nameComponents.first, nameComponents.second);
-      }
-    | "alter" "feed" nameComponents = QualifiedName() "set" configuration = Configuration()
-      {
-        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.ALTER, nameComponents.first, nameComponents.second, configuration);
+        stmt = new DisconnectFeedStatement(feedNameComponents, datasetNameComponents);
       }
   )
     {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index 60cf6e5..8481817 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -17,28 +17,20 @@
 public class DatasetConfig {
 
     /*
-     * We have three kinds of datasets. INTERNAL: A dataset with data persisted
+     * We have two kinds of datasets. INTERNAL: A dataset with data persisted
      * in ASTERIX storage. The dataset is populated either using a load
      * statement or using insert statement. EXTERNAL: A dataset with data
      * residing outside ASTERIX. As such ASTERIX does not maintain any indexes
      * on the data. The data for the dataset is fetched as and when required
-     * from an external data source using an adapter. FEED : A dataset that can
-     * be considered as a hybrid of INTERNAL and EXTERNAL dataset. A FEED
-     * dataset is INTERNAL in the sense that the data is persisted within
-     * ASTERIX storage and has associated indexes that are maintained by
-     * ASTERIX. However the dataset is initially populated with data fetched
-     * from an external datasource using an adapter, in a manner similar to an
-     * EXTERNAL dataset. A FEED dataset continuously receives data from the
-     * associated adapter.
+     * from an external data source using an adapter.
      */
     public enum DatasetType {
         INTERNAL,
-        EXTERNAL,
-        FEED
+        EXTERNAL
     }
 
     public enum IndexType {
-        BTREE,        
+        BTREE,
         RTREE,
         SINGLE_PARTITION_WORD_INVIX,
         SINGLE_PARTITION_NGRAM_INVIX,
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
index 27750eb..2b3ce0d 100644
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
@@ -20,9 +20,9 @@
 	message-text: string
 }
 
-
-create feed dataset TwitterFirehoseFeed(TweetMessageType)
-using twitter_firehose
-(("duration"="5"),("tps"="50"))
+create dataset Tweets(TweetMessageType)
 primary key tweetid;
 
+create feed  TwitterFirehose
+using twitter_firehose
+(("duration"="5"),("tps"="50"));
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
index 382425f..64dbf25 100644
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
@@ -1,4 +1,3 @@
 use dataverse feeds;
 
-begin feed TwitterFirehoseFeed;
-
+connect feed TwitterFirehose to dataset Tweets;
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
index 7244f68..d03b9fe 100644
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
@@ -1,6 +1,6 @@
 use dataverse feeds;
 
-let $numTuples:=count(for $x in dataset TwitterFirehoseFeed
+let $numTuples:=count(for $x in dataset Tweets
 return $x)
 let $result:=if($numTuples > 225)
 then 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
index ee476c4..ca0b6f1 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
@@ -26,13 +26,14 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Library;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 
 /**
  * Caches metadata entities such that the MetadataManager does not have to
@@ -57,12 +58,14 @@
     // Key is adapter dataverse. Key of value map is the adapter name  
     protected final Map<String, Map<String, DatasourceAdapter>> adapters = new HashMap<String, Map<String, DatasourceAdapter>>();
     // Key is FeedId   
-    protected final Map<FeedId, FeedActivity> feedActivity = new HashMap<FeedId, FeedActivity>();
+    protected final Map<FeedConnectionId, FeedActivity> feedActivity = new HashMap<FeedConnectionId, FeedActivity>();
 
     // Key is DataverseName, Key of the value map is the Policy name   
     protected final Map<String, Map<String, FeedPolicy>> feedPolicies = new HashMap<String, Map<String, FeedPolicy>>();
-    // Key is library dataverse. Key of value map is the library  
+    // Key is library dataverse. Key of value map is the library name
     protected final Map<String, Map<String, Library>> libraries = new HashMap<String, Map<String, Library>>();
+    // Key is library dataverse. Key of value map is the feed name  
+    protected final Map<String, Map<String, Feed>> feeds = new HashMap<String, Map<String, Feed>>();
 
     // Atomically executes all metadata operations in ctx's log.
     public void commit(MetadataTransactionContext ctx) {
@@ -199,31 +202,38 @@
                     synchronized (datatypes) {
                         synchronized (functions) {
                             synchronized (adapters) {
-                                synchronized (feedActivity) {
-                                    datasets.remove(dataverse.getDataverseName());
-                                    indexes.remove(dataverse.getDataverseName());
-                                    datatypes.remove(dataverse.getDataverseName());
-                                    adapters.remove(dataverse.getDataverseName());
-                                    List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
-                                    for (FunctionSignature signature : functions.keySet()) {
-                                        if (signature.getNamespace().equals(dataverse.getDataverseName())) {
-                                            markedFunctionsForRemoval.add(signature);
-                                        }
-                                    }
-                                    for (FunctionSignature signature : markedFunctionsForRemoval) {
-                                        functions.remove(signature);
-                                    }
-                                    List<FeedId> feedActivitiesMarkedForRemoval = new ArrayList<FeedId>();
-                                    for (FeedId fid : feedActivity.keySet()) {
-                                        if (fid.getDataverse().equals(dataverse.getDataverseName())) {
-                                            feedActivitiesMarkedForRemoval.add(fid);
-                                        }
-                                    }
-                                    for (FeedId fid : feedActivitiesMarkedForRemoval) {
-                                        feedActivity.remove(fid);
-                                    }
+                                synchronized (libraries) {
+                                    synchronized (feedActivity) {
+                                        synchronized (feeds) {
+                                            datasets.remove(dataverse.getDataverseName());
+                                            indexes.remove(dataverse.getDataverseName());
+                                            datatypes.remove(dataverse.getDataverseName());
+                                            adapters.remove(dataverse.getDataverseName());
+                                            List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
+                                            for (FunctionSignature signature : functions.keySet()) {
+                                                if (signature.getNamespace().equals(dataverse.getDataverseName())) {
+                                                    markedFunctionsForRemoval.add(signature);
+                                                }
+                                            }
+                                            for (FunctionSignature signature : markedFunctionsForRemoval) {
+                                                functions.remove(signature);
+                                            }
+                                            List<FeedConnectionId> feedActivitiesMarkedForRemoval = new ArrayList<FeedConnectionId>();
+                                            for (FeedConnectionId fid : feedActivity.keySet()) {
+                                                if (fid.getDataverse().equals(dataverse.getDataverseName())) {
+                                                    feedActivitiesMarkedForRemoval.add(fid);
+                                                }
+                                            }
+                                            for (FeedConnectionId fid : feedActivitiesMarkedForRemoval) {
+                                                feedActivity.remove(fid);
+                                            }
 
-                                    return dataverses.remove(dataverse.getDataverseName());
+                                            libraries.remove(dataverse.getDataverseName());
+                                            feeds.remove(dataverse.getDataverseName());
+
+                                            return dataverses.remove(dataverse.getDataverseName());
+                                        }
+                                    }
                                 }
                             }
                         }
@@ -461,7 +471,7 @@
 
     public Object addFeedActivityIfNotExists(FeedActivity fa) {
         synchronized (feedActivity) {
-            FeedId fid = new FeedId(fa.getDataverseName(), fa.getDatasetName());
+            FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
             if (!feedActivity.containsKey(fid)) {
                 feedActivity.put(fid, fa);
             }
@@ -471,7 +481,7 @@
 
     public Object dropFeedActivity(FeedActivity fa) {
         synchronized (feedActivity) {
-            FeedId fid = new FeedId(fa.getDataverseName(), fa.getDatasetName());
+            FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
             return feedActivity.remove(fid);
         }
     }
@@ -500,4 +510,19 @@
             return null;
         }
     }
+
+    public Object addFeedIfNotExists(Feed feed) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    public Object dropFeed(Feed feed) {
+        synchronized (feeds) {
+            Map<String, Feed> feedsInDataverse = feeds.get(feed.getDataverseName());
+            if (feedsInDataverse != null) {
+                return feedsInDataverse.remove(feed.getFeedName());
+            }
+            return null;
+        }
+    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 9132c63..0ba5c13 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
@@ -39,7 +40,7 @@
 import edu.uci.ics.asterix.metadata.entities.Library;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 
@@ -607,7 +608,7 @@
     }
 
     @Override
-    public void registerFeedActivity(MetadataTransactionContext ctx, FeedId feedId, FeedActivity feedActivity)
+    public void registerFeedActivity(MetadataTransactionContext ctx, FeedConnectionId feedId, FeedActivity feedActivity)
             throws MetadataException {
         try {
             metadataNode.registerFeedActivity(ctx.getJobId(), feedId, feedActivity);
@@ -617,13 +618,12 @@
     }
 
     @Override
-    public FeedActivity getRecentFeedActivity(MetadataTransactionContext ctx, String dataverseName, String datasetName,
+    public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
             FeedActivityType... feedActivityTypes) throws MetadataException {
 
         FeedActivity feedActivity = null;
         try {
-            feedActivity = metadataNode.getRecentFeedActivity(ctx.getJobId(), new FeedId(dataverseName, datasetName),
-                    feedActivityTypes);
+            feedActivity = metadataNode.getRecentFeedActivity(ctx.getJobId(), feedId, feedActivityTypes);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -712,10 +712,53 @@
     }
 
     @Override
-    public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx) throws MetadataException {
+    public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx, String dataverse, String dataset)
+            throws MetadataException {
         List<FeedActivity> feedActivities = null;
         try {
-            feedActivities = metadataNode.getActiveFeeds(ctx.getJobId());
+            feedActivities = metadataNode.getActiveFeeds(ctx.getJobId(), dataverse, dataset);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return feedActivities;
+    }
+
+    @Override
+    public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+        Feed feed = null;
+        try {
+            feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return feed;
+    }
+
+    @Override
+    public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+        try {
+            metadataNode.dropFeed(ctx.getJobId(), dataverse, feedName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.dropFeed(dataverse, feedName);
+    }
+
+    @Override
+    public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException {
+        try {
+            metadataNode.addFeed(ctx.getJobId(), feed);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.addFeed(feed);
+    }
+
+    public List<FeedActivity> getConnectFeedActivitiesForFeed(MetadataTransactionContext ctx, String dataverse,
+            String feedName) throws MetadataException {
+        List<FeedActivity> feedActivities = null;
+        try {
+            feedActivities = metadataNode.getDatasetsServedByFeed(ctx.getJobId(), dataverse, feedName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index ff3909a..ade794e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -18,9 +18,9 @@
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
@@ -45,6 +45,7 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
@@ -60,13 +61,14 @@
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedActivityTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeTupleTranslator;
 import edu.uci.ics.asterix.metadata.feeds.FeedActivityIdFactory;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.valueextractors.DatasetNameValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.DatatypeNameValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
@@ -82,6 +84,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -174,7 +177,7 @@
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
             ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
             insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
-            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 // Add the primary index for the dataset.
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
@@ -312,6 +315,7 @@
     @Override
     public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
         try {
+
             List<Dataset> dataverseDatasets;
             Dataset ds;
             dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
@@ -320,13 +324,6 @@
                 for (int i = 0; i < dataverseDatasets.size(); i++) {
                     ds = dataverseDatasets.get(i);
                     dropDataset(jobId, dataverseName, ds.getDatasetName());
-                    if (ds.getDatasetDetails().getDatasetType().equals(DatasetType.FEED)) {
-                        String ngName = ds.getDataverseName() + ":" + ds.getDatasetName();
-                        NodeGroup ng = getNodeGroup(jobId, ngName);
-                        if (ng != null) {
-                            dropNodegroup(jobId, ngName);
-                        }
-                    }
                 }
             }
             List<Datatype> dataverseDatatypes;
@@ -360,6 +357,17 @@
                 }
             }
 
+            List<Feed> dataverseFeeds;
+            Feed feed;
+            dataverseFeeds = getDataverseFeeds(jobId, dataverseName);
+            if (dataverseFeeds != null && dataverseFeeds.size() > 0) {
+                // Drop all datasets in this dataverse.
+                for (int i = 0; i < dataverseFeeds.size(); i++) {
+                    feed = dataverseFeeds.get(i);
+                    dropFeed(jobId, dataverseName, feed.getFeedName());
+                }
+            }
+
             // Delete the dataverse entry from the 'dataverse' dataset.
             ITupleReference searchKey = createTuple(dataverseName);
             // As a side effect, acquires an S lock on the 'dataverse' dataset
@@ -403,7 +411,7 @@
             }
 
             // Delete entry from secondary index 'group'.
-            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
                 // Searches the index for the tuple to be deleted. Acquires an S
@@ -431,7 +439,7 @@
             }
 
             // Delete entry(s) from the 'indexes' dataset.
-            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
                 if (datasetIndexes != null) {
                     for (Index index : datasetIndexes) {
@@ -659,6 +667,20 @@
         }
     }
 
+    @Override
+    public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName);
+            FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
+            IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
+            List<Feed> results = new ArrayList<Feed>();
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+            return results;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
     public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
@@ -1238,8 +1260,8 @@
     }
 
     @Override
-    public void registerFeedActivity(JobId jobId, FeedId feedId, FeedActivity feedActivity) throws MetadataException,
-            RemoteException {
+    public void registerFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException, RemoteException {
         try {
             if (!FeedActivityIdFactory.isInitialized()) {
                 initializeFeedActivityIdFactory(jobId);
@@ -1255,10 +1277,11 @@
     }
 
     @Override
-    public FeedActivity getRecentFeedActivity(JobId jobId, FeedId feedId, FeedActivityType... activityType)
+    public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivityType... activityType)
             throws MetadataException, RemoteException {
         try {
-            ITupleReference searchKey = createTuple(feedId.getDataverse(), feedId.getDataset());
+            ITupleReference searchKey = createTuple(feedId.getDataverse(), feedId.getFeedName(),
+                    feedId.getDatasetName());
             FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(false);
             List<FeedActivity> results = new ArrayList<FeedActivity>();
             IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
@@ -1343,7 +1366,8 @@
     }
 
     @Override
-    public List<FeedActivity> getActiveFeeds(JobId jobId) throws MetadataException, RemoteException {
+    public List<FeedActivity> getActiveFeeds(JobId jobId, String dataverse, String dataset) throws MetadataException,
+            RemoteException {
         List<FeedActivity> activeFeeds = new ArrayList<FeedActivity>();
         try {
             ITupleReference searchKey = createTuple();
@@ -1352,28 +1376,122 @@
             IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
                     tupleReaderWriter);
             searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
-            Map<FeedId, FeedActivity> initiatedFeeds = new HashMap<FeedId, FeedActivity>();
-            FeedId fid = null;
+            Collections.sort(results); // recent activity first
+            FeedConnectionId fid = null;
+            Set<FeedConnectionId> terminatedFeeds = new HashSet<FeedConnectionId>();
             for (FeedActivity fa : results) {
+                if (dataset != null
+                        && (!fa.getDataverseName().equals(dataverse) || !dataset.equals(fa.getDatasetName()))) {
+                    continue;
+                }
+                fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
                 switch (fa.getActivityType()) {
+                    case FEED_RESUME:
                     case FEED_BEGIN:
-                        fid = new FeedId(fa.getDataverseName(), fa.getDatasetName());
-                        initiatedFeeds.put(fid, fa);
-                        break;
-                    case FEED_FAILURE:
+                        if (!terminatedFeeds.contains(fid)) {
+                            activeFeeds.add(fa);
+                        }
                         break;
                     case FEED_END:
-                        fid = new FeedId(fa.getDataverseName(), fa.getDatasetName());
-                        initiatedFeeds.remove(fid);
+                        terminatedFeeds.add(fid);
                         break;
                 }
             }
-            for (FeedActivity fa : initiatedFeeds.values()) {
-                activeFeeds.add(fa);
-            }
             return activeFeeds;
         } catch (Exception e) {
             throw new MetadataException(e);
         }
     }
+
+    @Override
+    public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException {
+        try {
+            // Insert into the 'Feed' dataset.
+            FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(true);
+            ITupleReference feedTuple = tupleReaderWriter.getTupleFromMetadataEntity(feed);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
+
+        } catch (BTreeDuplicateKeyException e) {
+            throw new MetadataException("A feed with this name " + feed.getFeedName()
+                    + " already exists in dataverse '" + feed.getDataverseName() + "'.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+        try {
+            ITupleReference searchKey = createTuple(dataverse, feedName);
+            FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
+            List<Feed> results = new ArrayList<Feed>();
+            IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+            if (!results.isEmpty()) {
+                return results.get(0);
+            }
+            return null;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+
+        try {
+            ITupleReference searchKey = createTuple(dataverse, feedName);
+            // Searches the index for the tuple to be deleted. Acquires an S
+            // lock on the 'nodegroup' dataset.
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
+            // TODO: Change this to be a BTree specific exception, e.g.,
+            // BTreeKeyDoesNotExistException.
+        } catch (TreeIndexException e) {
+            throw new MetadataException("Cannot drop feed '" + feedName + "' because it doesn't exist", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverse, String feedName)
+            throws MetadataException, RemoteException {
+        List<FeedActivity> feedActivities = new ArrayList<FeedActivity>();
+        try {
+            ITupleReference searchKey = createTuple(dataverse, feedName);
+            FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(false);
+            List<FeedActivity> results = new ArrayList<FeedActivity>();
+            IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+                    tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+
+            if (!results.isEmpty()) {
+                Collections.sort(results); // most recent feed activity
+                Set<String> terminatedDatasets = new HashSet<String>();
+                Set<String> activeDatasets = new HashSet<String>();
+
+                for (FeedActivity result : results) {
+                    switch (result.getFeedActivityType()) {
+                        case FEED_BEGIN:
+                        case FEED_RESUME:
+                            if (!terminatedDatasets.contains(result.getDatasetName())) {
+                                feedActivities.add(result);
+                                activeDatasets.add(result.getDatasetName());
+                            }
+                            break;
+                        case FEED_END:
+                            if (!activeDatasets.contains(result.getDatasetName())) {
+                                terminatedDatasets.add(result.getDatasetName());
+                            }
+                            break;
+                    }
+
+                }
+            }
+            return feedActivities;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
index 93ae505..48fbdde 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
@@ -16,6 +16,7 @@
 package edu.uci.ics.asterix.metadata;
 
 import java.util.ArrayList;
+import java.util.Map;
 
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.JobId;
@@ -24,6 +25,7 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
@@ -207,6 +209,24 @@
         return opLog;
     }
 
+    public void addFeedPolicy(FeedPolicy feedPolicy) {
+        droppedCache.dropFeedPolicy(feedPolicy);
+        logAndApply(new MetadataLogicalOperation(feedPolicy, true));
+
+    }
+
+    public void addFeed(Feed feed) {
+        droppedCache.dropFeed(feed);
+        logAndApply(new MetadataLogicalOperation(feed, true));
+
+    }
+
+    public void dropFeed(String dataverse, String feedName) {
+        Feed feed = new Feed(dataverse, feedName, null, null, null);
+        droppedCache.addFeedIfNotExists(feed);
+        logAndApply(new MetadataLogicalOperation(feed, false));
+    }
+
     @Override
     public void clear() {
         super.clear();
@@ -214,9 +234,4 @@
         opLog.clear();
     }
 
-    public void addFeedPolicy(FeedPolicy feedPolicy) {
-        droppedCache.dropFeedPolicy(feedPolicy);
-        logAndApply(new MetadataLogicalOperation(feedPolicy, true));
-
-    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index 8befe1f..d001119 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -16,7 +16,6 @@
 package edu.uci.ics.asterix.metadata.api;
 
 import java.rmi.RemoteException;
-import java.util.Collection;
 import java.util.List;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
@@ -27,6 +26,7 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
@@ -35,7 +35,7 @@
 import edu.uci.ics.asterix.metadata.entities.Library;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 
 /**
  * A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -448,11 +448,35 @@
 
     /**
      * @param ctx
+     * @param feed
+     * @throws MetadataException
+     */
+    public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param dataverse
+     * @param feedName
+     * @return
+     * @throws MetadataException
+     */
+    public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param dataverse
+     * @param feedName
+     * @throws MetadataException
+     */
+    public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+
+    /**
+     * @param ctx
      * @param feedId
      * @param feedActivity
      * @throws MetadataException
      */
-    public void registerFeedActivity(MetadataTransactionContext ctx, FeedId feedId, FeedActivity feedActivity)
+    public void registerFeedActivity(MetadataTransactionContext ctx, FeedConnectionId feedId, FeedActivity feedActivity)
             throws MetadataException;
 
     /**
@@ -462,7 +486,7 @@
      * @return
      * @throws MetadataException
      */
-    public FeedActivity getRecentFeedActivity(MetadataTransactionContext ctx, String dataverseName, String datasetName,
+    public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
             FeedActivityType... activityTypeFilter) throws MetadataException;
 
     /**
@@ -484,10 +508,16 @@
 
     /**
      * @param ctx
+     * @param dataverse
+     * @param dataset
      * @return
      * @throws MetadataException
      */
-    public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx) throws MetadataException;
+    public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx, String dataverse, String dataset)
+            throws MetadataException;
+
+    public List<FeedActivity> getConnectFeedActivitiesForFeed(MetadataTransactionContext ctx, String dataverse,
+            String dataset) throws MetadataException;
 
     public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index 3c2e241..1dcdb01 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
@@ -36,7 +37,7 @@
 import edu.uci.ics.asterix.metadata.entities.Library;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 
 /**
  * A metadata node stores metadata in its local storage structures (currently
@@ -484,8 +485,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public FeedActivity getRecentFeedActivity(JobId jobId, FeedId feedId, FeedActivityType... feedActivityFilter)
-            throws MetadataException, RemoteException;
+    public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId,
+            FeedActivityType... feedActivityFilter) throws MetadataException, RemoteException;
 
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
 
@@ -500,13 +501,40 @@
 
     /**
      * @param jobId
+     * @param feed
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
+     * @param dataverse
+     * @param feedName
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
+     * @param dataverse
+     * @param feedName
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param feedId
      *            A unique id for the feed
      * @param feedActivity
      */
-    public void registerFeedActivity(JobId jobId, FeedId feedId, FeedActivity feedActivity) throws MetadataException,
-            RemoteException;
+    public void registerFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -529,11 +557,14 @@
 
     /**
      * @param jobId
+     * @param dataverse
+     * @param dataset
      * @return
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<FeedActivity> getActiveFeeds(JobId jobId) throws MetadataException, RemoteException;
+    public List<FeedActivity> getActiveFeeds(JobId jobId, String dataverse, String dataset) throws MetadataException,
+            RemoteException;
 
     /**
      * Removes a library , acquiring local locks on behalf of the given
@@ -593,4 +624,23 @@
     public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException;
 
+    /**
+     * @param jobId
+     * @param dataverseName
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
+     * @param dataverseName
+     * @param deedName
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverseName, String deedName)
+            throws MetadataException, RemoteException;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 262b957..2d60c41 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -120,8 +120,9 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET,
-                MetadataPrimaryIndexes.FEED_POLICY_DATASET, MetadataPrimaryIndexes.LIBRARY_DATASET };
+                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
+                MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
+                MetadataPrimaryIndexes.LIBRARY_DATASET };
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -318,9 +319,10 @@
                 "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory",
                 "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory",
                 "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory",
+                "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory",
                 "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory",
                 "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory",
-                "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory"};
+                "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory" };
         DatasourceAdapter adapter;
         for (String adapterClassName : builtInAdapterClassNames) {
             adapter = getAdapter(adapterClassName);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 12fe446..b8501b3 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -34,6 +34,7 @@
     public static IMetadataIndex FUNCTION_DATASET;
     public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
     public static IMetadataIndex LIBRARY_DATASET;
+    public static IMetadataIndex FEED_DATASET;
     public static IMetadataIndex FEED_ACTIVITY_DATASET;
     public static IMetadataIndex FEED_POLICY_DATASET;
 
@@ -47,8 +48,9 @@
     public static final int FUNCTION_DATASET_ID = 7;
     public static final int DATASOURCE_ADAPTER_DATASET_ID = 8;
     public static final int LIBRARY_DATASET_ID = 9;
-    public static final int FEED_ACTIVITY_DATASET_ID = 10;
-    public static final int FEED_POLICY_DATASET_ID = 11;
+    public static final int FEED_DATASET_ID = 10;
+    public static final int FEED_ACTIVITY_DATASET_ID = 11;
+    public static final int FEED_POLICY_DATASET_ID = 12;
 
     public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
 
@@ -100,10 +102,14 @@
                 MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_DATASET_ID, true, new int[] { 0,
                         1 });
 
-        FEED_ACTIVITY_DATASET = new MetadataIndex("FeedActivity", null, 4, new IAType[] { BuiltinType.ASTRING,
-                BuiltinType.ASTRING, BuiltinType.AINT32 },
-                new String[] { "DataverseName", "DatasetName", "ActivityId" }, 0,
-                MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE, FEED_ACTIVITY_DATASET_ID, true, new int[] { 0, 1, 2 });
+        FEED_DATASET = new MetadataIndex("Feed", null, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+                new String[] { "DataverseName", "FeedName" }, 0, MetadataRecordTypes.FEED_RECORDTYPE, FEED_DATASET_ID,
+                true, new int[] { 0, 1 });
+
+        FEED_ACTIVITY_DATASET = new MetadataIndex("FeedActivity", null, 5, new IAType[] { BuiltinType.ASTRING,
+                BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 }, new String[] { "DataverseName",
+                "FeedName", "DatasetName", "ActivityId" }, 0, MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE,
+                FEED_ACTIVITY_DATASET_ID, true, new int[] { 0, 1, 2, 3 });
 
         LIBRARY_DATASET = new MetadataIndex("Library", null, 3,
                 new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" }, 0,
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 3acf97f..71b1f0d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -47,6 +47,8 @@
     public static ARecordType NODEGROUP_RECORDTYPE;
     public static ARecordType FUNCTION_RECORDTYPE;
     public static ARecordType DATASOURCE_ADAPTER_RECORDTYPE;
+    public static ARecordType FEED_RECORDTYPE;
+    public static ARecordType FEED_ADAPTOR_CONFIGURATION_RECORDTYPE;
     public static ARecordType FEED_ACTIVITY_RECORDTYPE;
     public static ARecordType FEED_POLICY_RECORDTYPE;
     public static ARecordType POLICY_PARAMS_RECORDTYPE;
@@ -82,6 +84,8 @@
             NODEGROUP_RECORDTYPE = createNodeGroupRecordType();
             FUNCTION_RECORDTYPE = createFunctionRecordType();
             DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
+            FEED_RECORDTYPE = createFeedRecordType();
+            FEED_ADAPTOR_CONFIGURATION_RECORDTYPE = createPropertiesRecordType();
             FEED_ACTIVITY_DETAILS_RECORDTYPE = createPropertiesRecordType();
             FEED_ACTIVITY_RECORDTYPE = createFeedActivityRecordType();
             FEED_POLICY_RECORDTYPE = createFeedPolicyRecordType();
@@ -190,15 +194,14 @@
     public static final int DATASET_ARECORD_DATASETTYPE_FIELD_INDEX = 3;
     public static final int DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX = 4;
     public static final int DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX = 5;
-    public static final int DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX = 6;
-    public static final int DATASET_ARECORD_HINTS_FIELD_INDEX = 7;
-    public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 8;
-    public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 9;
-    public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 10;
+    public static final int DATASET_ARECORD_HINTS_FIELD_INDEX = 6;
+    public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 7;
+    public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 8;
+    public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 9;
 
     private static final ARecordType createDatasetRecordType() throws AsterixException {
         String[] fieldNames = { "DataverseName", "DatasetName", "DataTypeName", "DatasetType", "InternalDetails",
-                "ExternalDetails", "FeedDetails", "Hints", "Timestamp", "DatasetId", "PendingOp" };
+                "ExternalDetails", "Hints", "Timestamp", "DatasetId", "PendingOp" };
 
         List<IAType> internalRecordUnionList = new ArrayList<IAType>();
         internalRecordUnionList.add(BuiltinType.ANULL);
@@ -210,16 +213,11 @@
         externalRecordUnionList.add(EXTERNAL_DETAILS_RECORDTYPE);
         AUnionType externalRecordUnion = new AUnionType(externalRecordUnionList, null);
 
-        List<IAType> feedRecordUnionList = new ArrayList<IAType>();
-        feedRecordUnionList.add(BuiltinType.ANULL);
-        feedRecordUnionList.add(FEED_DETAILS_RECORDTYPE);
-        AUnionType feedRecordUnion = new AUnionType(feedRecordUnionList, null);
-
         AUnorderedListType unorderedListOfHintsType = new AUnorderedListType(DATASET_HINTS_RECORDTYPE, null);
 
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
-                internalRecordUnion, externalRecordUnion, feedRecordUnion, unorderedListOfHintsType,
-                BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.AINT32 };
+                internalRecordUnion, externalRecordUnion, unorderedListOfHintsType, BuiltinType.ASTRING,
+                BuiltinType.AINT32, BuiltinType.AINT32 };
         return new ARecordType("DatasetRecordType", fieldNames, fieldTypes, true);
     }
 
@@ -381,20 +379,48 @@
     // Helper constants for accessing fields in an ARecord of type
     // FeedActivityRecordType.
     public static final int FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
-    public static final int FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX = 1;
-    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX = 2;
-    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX = 3;
-    public static final int FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX = 4;
-    public static final int FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX = 5;
+    public static final int FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX = 1;
+    public static final int FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX = 2;
+    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX = 3;
+    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX = 4;
+    public static final int FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX = 5;
+    public static final int FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX = 6;
 
     private static ARecordType createFeedActivityRecordType() throws AsterixException {
         AUnorderedListType unorderedPropertyListType = new AUnorderedListType(FEED_ACTIVITY_DETAILS_RECORDTYPE, null);
-        String[] fieldNames = { "DataverseName", "DatasetName", "ActivityId", "ActivityType", "Details", "Timestamp" };
-        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.ASTRING,
-                unorderedPropertyListType, BuiltinType.ASTRING };
+        String[] fieldNames = { "DataverseName", "FeedName", "DatasetName", "ActivityId", "ActivityType", "Details",
+                "Timestamp" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+                BuiltinType.ASTRING, unorderedPropertyListType, BuiltinType.ASTRING };
         return new ARecordType("FeedActivityRecordType", fieldNames, fieldTypes, true);
     }
 
+    public static final int FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+    public static final int FEED_ARECORD_FEED_NAME_FIELD_INDEX = 1;
+    public static final int FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX = 2;
+    public static final int FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX = 3;
+    public static final int FEED_ARECORD_FUNCTION_FIELD_INDEX = 4;
+    public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 5;
+
+    private static ARecordType createFeedRecordType() throws AsterixException {
+
+        AUnorderedListType unorderedAdaptorPropertyListType = new AUnorderedListType(
+                DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
+
+        List<IAType> feedFunctionUnionList = new ArrayList<IAType>();
+        feedFunctionUnionList.add(BuiltinType.ANULL);
+        feedFunctionUnionList.add(BuiltinType.ASTRING);
+        AUnionType feedFunctionUnion = new AUnionType(feedFunctionUnionList, null);
+
+        String[] fieldNames = { "DataverseName", "FeedName", "AdaptorName", "AdaptorConfiguration", "Function",
+                "Timestamp" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                unorderedAdaptorPropertyListType, feedFunctionUnion, BuiltinType.ASTRING };
+
+        return new ARecordType("FeedRecordType", fieldNames, fieldTypes, true);
+
+    }
+
     public static final int LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
     public static final int LIBRARY_ARECORD_NAME_FIELD_INDEX = 1;
     public static final int LIBRARY_ARECORD_TIMESTAMP_FIELD_INDEX = 2;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
index 315c5e9..c33c21f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
@@ -56,5 +56,6 @@
                 BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
                 "NestedDatatypeName", "TopDatatypeName" }, 2, null, MetadataPrimaryIndexes.DATATYPE_DATASET_ID, false,
                 new int[] { 0, 2 });
+
     }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index da15cfe..fa0311d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -222,8 +222,8 @@
 
         File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
         Dataset dataset = findDataset(datasetName);
-        if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
-            throw new AlgebricksException("Not an internal or feed dataset");
+        if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+            throw new AlgebricksException("Not an internal dataset");
         }
         InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
         List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 077b3f1..874a838 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.asterix.metadata.declared;
 
-import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -23,11 +22,6 @@
 import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
@@ -35,7 +29,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.AsterixNodeGroupDomain;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -47,146 +40,55 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 
-public class AqlDataSource implements IDataSource<AqlSourceId> {
+public abstract class AqlDataSource implements IDataSource<AqlSourceId> {
 
     private AqlSourceId id;
-    private Dataset dataset;
-    private IAType[] schemaTypes;
-    private INodeDomain domain;
+    private String datasourceDataverse;
+    private String datasourceName;
     private AqlDataSourceType datasourceType;
+    protected IAType[] schemaTypes;
+    protected INodeDomain domain;
     private Map<String, Serializable> properties = new HashMap<String, Serializable>();
 
     public enum AqlDataSourceType {
-        INTERNAL,
-        FEED,
-        EXTERNAL,
-        EXTERNAL_FEED,
+        INTERNAL_DATASET,
+        EXTERNAL_DATASET,
+        FEED
     }
 
-    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType datasourceType)
-            throws AlgebricksException {
+    public AqlDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
+            AqlDataSourceType datasourceType) throws AlgebricksException {
         this.id = id;
-        this.dataset = dataset;
+        this.datasourceDataverse = datasourceDataverse;
+        this.datasourceName = datasourceName;
         this.datasourceType = datasourceType;
-        try {
-            switch (datasourceType) {
-                case FEED:
-                    initFeedDataset(itemType, dataset);
-                case INTERNAL: {
-                    initInternalDataset(itemType);
-                    break;
-                }
-                case EXTERNAL_FEED:
-                case EXTERNAL: {
-                    initExternalDataset(itemType);
-                    break;
-                }
-                default: {
-                    throw new IllegalArgumentException();
-                }
-            }
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
     }
 
-    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
-        this.id = id;
-        this.dataset = dataset;
-        try {
-            switch (dataset.getDatasetType()) {
-                case FEED:
-                    datasourceType = AqlDataSourceType.FEED;
-                    initFeedDataset(itemType, dataset);
-                    break;
-                case INTERNAL:
-                    datasourceType = AqlDataSourceType.INTERNAL;
-                    initInternalDataset(itemType);
-                    break;
-                case EXTERNAL: {
-                    datasourceType = AqlDataSourceType.EXTERNAL;
-                    initExternalDataset(itemType);
-                    break;
-                }
-                default: {
-                    throw new IllegalArgumentException();
-                }
-            }
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
+    public String getDatasourceDataverse() {
+        return datasourceDataverse;
     }
 
-    // TODO: Seems like initFeedDataset() could simply call this method.
-    private void initInternalDataset(IAType itemType) throws IOException {
-        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        ARecordType recordType = (ARecordType) itemType;
-        int n = partitioningKeys.size();
-        schemaTypes = new IAType[n + 1];
-        for (int i = 0; i < n; i++) {
-            schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
-        }
-        schemaTypes[n] = itemType;
-        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+    public String getDatasourceName() {
+        return datasourceName;
     }
 
-    private void initFeedDataset(IAType itemType, Dataset dataset) throws IOException {
-        if (dataset.getDatasetDetails() instanceof ExternalDatasetDetails) {
-            initExternalDataset(itemType);
-        } else {
-            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-            int n = partitioningKeys.size();
-            schemaTypes = new IAType[n + 1];
-            ARecordType recordType = (ARecordType) itemType;
-            for (int i = 0; i < n; i++) {
-                schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
-            }
-            schemaTypes[n] = itemType;
-            domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
-        }
-    }
+    public abstract IAType[] getSchemaTypes();
 
-    private void initExternalDataset(IAType itemType) {
-        schemaTypes = new IAType[1];
-        schemaTypes[0] = itemType;
-        INodeDomain domainForExternalData = new INodeDomain() {
-            @Override
-            public Integer cardinality() {
-                return null;
-            }
-
-            @Override
-            public boolean sameAs(INodeDomain domain) {
-                return domain == this;
-            }
-        };
-        domain = domainForExternalData;
-    }
+    public abstract INodeDomain getDomain();
 
     @Override
     public AqlSourceId getId() {
         return id;
     }
 
-    public Dataset getDataset() {
-        return dataset;
-    }
-
-    @Override
-    public IAType[] getSchemaTypes() {
-        return schemaTypes;
-    }
-
     @Override
     public String toString() {
         return id.toString();
-        // return "AqlDataSource(\"" + id.getDataverseName() + "/" +
-        // id.getDatasetName() + "\")";
     }
 
     @Override
     public IDataSourcePropertiesProvider getPropertiesProvider() {
-        return new AqlDataSourcePartitioningProvider(dataset.getDatasetType(), domain);
+        return new AqlDataSourcePartitioningProvider(datasourceType, domain);
     }
 
     @Override
@@ -205,21 +107,24 @@
 
         private INodeDomain domain;
 
-        private DatasetType datasetType;
+        private AqlDataSourceType aqlDataSourceType;
 
-        public AqlDataSourcePartitioningProvider(DatasetType datasetType, INodeDomain domain) {
-            this.datasetType = datasetType;
+        public AqlDataSourcePartitioningProvider(AqlDataSourceType datasetSourceType, INodeDomain domain) {
+            this.aqlDataSourceType = datasetSourceType;
             this.domain = domain;
         }
 
         @Override
         public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
-            switch (datasetType) {
-                case EXTERNAL: {
+            IPhysicalPropertiesVector propsVector = null;
+
+            switch (aqlDataSourceType) {
+                case EXTERNAL_DATASET: {
                     IPartitioningProperty pp = new RandomPartitioningProperty(domain);
                     List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
-                    return new StructuralPropertiesVector(pp, propsLocal);
+                    propsVector = new StructuralPropertiesVector(pp, propsLocal);
                 }
+
                 case FEED: {
                     int n = scanVariables.size();
                     IPartitioningProperty pp;
@@ -238,9 +143,11 @@
                         pp = new UnorderedPartitionedProperty(pvars, domain);
                     }
                     List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
-                    return new StructuralPropertiesVector(pp, propsLocal);
+                    propsVector = new StructuralPropertiesVector(pp, propsLocal);
+                    break;
                 }
-                case INTERNAL: {
+
+                case INTERNAL_DATASET: {
                     int n = scanVariables.size();
                     IPartitioningProperty pp;
                     if (n < 2) {
@@ -261,12 +168,15 @@
                     for (int i = 0; i < n - 1; i++) {
                         propsLocal.add(new LocalOrderProperty(new OrderColumn(scanVariables.get(i), OrderKind.ASC)));
                     }
-                    return new StructuralPropertiesVector(pp, propsLocal);
+                    propsVector = new StructuralPropertiesVector(pp, propsLocal);
                 }
+                    break;
+
                 default: {
                     throw new IllegalArgumentException();
                 }
             }
+            return propsVector;
         }
 
     }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 225110c..e34c735 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -45,25 +45,23 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
 import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Feed;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails.FeedState;
 import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
 import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
-import edu.uci.ics.asterix.metadata.feeds.FeedId;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
 import edu.uci.ics.asterix.metadata.feeds.FeedMessageOperatorDescriptor;
-import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
 import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
 import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory.SupportedOperation;
 import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
@@ -275,34 +273,20 @@
             List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
-        Dataset dataset;
         try {
-            dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataSource.getId().getDataverseName(), dataSource
-                    .getId().getDatasetName());
-
-            if (dataset == null) {
-                throw new AlgebricksException("Unknown dataset " + dataSource.getId().getDatasetName()
-                        + " in dataverse " + dataSource.getId().getDataverseName());
-            }
-            switch (dataset.getDatasetType()) {
+            switch (((AqlDataSource) dataSource).getDatasourceType()) {
                 case FEED:
-                    if (dataSource instanceof ExternalFeedDataSource) {
-                        return buildExternalDatasetScan(jobSpec, dataset, dataSource);
-                    } else {
-                        return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
-                                context, implConfig);
+                    return buildFeedIntakeRuntime(jobSpec, dataSource);
+                case INTERNAL_DATASET:
+                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataSource, context,
+                            implConfig);
 
-                    }
-                case INTERNAL: {
-                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
-                            context, implConfig);
-                }
-                case EXTERNAL: {
-                    return buildExternalDatasetScan(jobSpec, dataset, dataSource);
-                }
-                default: {
+                case EXTERNAL_DATASET:
+                    return buildExternalDatasetScan(jobSpec, dataSource);
+
+                default:
                     throw new IllegalArgumentException();
-                }
+
             }
         } catch (MetadataException e) {
             throw new AlgebricksException(e);
@@ -311,31 +295,26 @@
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig)
-            throws AlgebricksException, MetadataException {
+            IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig) throws AlgebricksException,
+            MetadataException {
         AqlSourceId asid = dataSource.getId();
         String dataverseName = asid.getDataverseName();
         String datasetName = asid.getDatasetName();
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
-        return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, false, dataset,
-                primaryIndex.getIndexName(), null, null, true, true, implConfig);
+        return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, false,
+                ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true, true,
+                implConfig);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
-            Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException, MetadataException {
+            IDataSource<AqlSourceId> dataSource) throws AlgebricksException, MetadataException {
+        Dataset dataset = ((DatasetDataSource) dataSource).getDataset();
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
                 .getDatatype();
-        switch (((AqlDataSource) dataSource).getDatasourceType()) {
-            case EXTERNAL_FEED:
-                return buildFeedIntakeRuntime(jobSpec, dataset, dataSource);
-            case EXTERNAL:
-                return buildExternalDataScannerRuntime(jobSpec, itemType,
-                        (ExternalDatasetDetails) dataset.getDatasetDetails(), NonTaggedDataFormat.INSTANCE);
-            default:
-                throw new IllegalStateException("Unknown aql datasource type: "
-                        + ((AqlDataSource) dataSource).getDatasourceType());
-        }
+        return buildExternalDataScannerRuntime(jobSpec, itemType, (ExternalDatasetDetails) dataset.getDatasetDetails(),
+                NonTaggedDataFormat.INSTANCE);
+
     }
 
     @SuppressWarnings("rawtypes")
@@ -427,17 +406,16 @@
 
     @SuppressWarnings("rawtypes")
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(JobSpecification jobSpec,
-            Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
+            IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
 
-        FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
         DatasourceAdapter adapterEntity;
         IAdapterFactory adapterFactory;
         IAType adapterOutputType;
         String adapterName;
         String adapterFactoryClassname;
-
+        FeedDataSource feedDataSource = (FeedDataSource) dataSource;
         try {
-            adapterName = datasetDetails.getAdapterFactory();
+            adapterName = feedDataSource.getFeed().getAdaptorName();
             adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
                     adapterName);
             if (adapterEntity != null) {
@@ -452,8 +430,7 @@
                 adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
             }
 
-            //Map<String, Object> configuration = this.wrapProperties(datasetDetails.getProperties());
-            Map<String, String> configuration = datasetDetails.getProperties();
+            Map<String, String> configuration = feedDataSource.getFeed().getAdaptorConfiguration();
 
             switch (adapterFactory.getAdapterType()) {
                 case TYPED:
@@ -461,9 +438,9 @@
                     ((ITypedAdapterFactory) adapterFactory).configure(configuration);
                     break;
                 case GENERIC:
-                    String outputTypeName = datasetDetails.getProperties().get("output-type-name");
-                    adapterOutputType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(),
-                            outputTypeName).getDatatype();
+                    String outputTypeName = configuration.get("output-type-name");
+                    adapterOutputType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            feedDataSource.getDatasourceDataverse(), outputTypeName).getDatatype();
                     ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType);
                     break;
                 default:
@@ -482,8 +459,9 @@
                 BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
 
         feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
-        FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(
-                dataset.getDataverseName(), dataset.getDatasetName()), adapterFactory, (ARecordType) adapterOutputType,
+        FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedConnectionId(
+                feedDataSource.getDatasourceDataverse(), feedDataSource.getDatasourceName(), feedDataSource
+                        .getFeedConnectionId().getDatasetName()), adapterFactory, (ARecordType) adapterOutputType,
                 feedDesc, feedPolicy.getProperties());
 
         AlgebricksPartitionConstraint constraint = null;
@@ -496,13 +474,12 @@
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
-            AqlMetadataProvider metadataProvider, JobSpecification jobSpec, FeedDatasetDetails datasetDetails,
-            String dataverse, String dataset, IFeedMessage feedMessage, FeedActivity feedActivity)
-            throws AlgebricksException {
+            AqlMetadataProvider metadataProvider, JobSpecification jobSpec, String dataverse, String feedName,
+            String dataset, IFeedMessage feedMessage, FeedActivity feedActivity) throws AlgebricksException {
         AlgebricksPartitionConstraint partitionConstraint = new AlgebricksAbsolutePartitionConstraint(feedActivity
                 .getFeedActivityDetails().get(FeedActivityDetails.INGEST_LOCATIONS).split(","));
-        FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
-                feedMessage);
+        FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, feedName,
+                dataset, feedMessage);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
     }
 
@@ -704,10 +681,11 @@
     public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
             throws AlgebricksException {
         AqlDataSource ads = findDataSource(dataSourceId);
-        Dataset dataset = ads.getDataset();
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+        if (ads.getDatasourceType() == AqlDataSourceType.EXTERNAL_DATASET) {
             throw new AlgebricksException("No index for external dataset " + dataSourceId);
         }
+        Dataset dataset = ((DatasetDataSource) ads).getDataset();
+
         try {
             String indexName = (String) indexId;
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -735,25 +713,24 @@
         }
         String tName = dataset.getItemTypeName();
         IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, aqlId.getDataverseName(), tName).getDatatype();
-        return new AqlDataSource(aqlId, dataset, itemType);
+        AqlDataSourceType datasourceType = dataset.getDatasetType().equals(DatasetType.EXTERNAL) ? AqlDataSourceType.EXTERNAL_DATASET
+                : AqlDataSourceType.INTERNAL_DATASET;
+        return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasetName(), itemType, datasourceType);
     }
 
     @Override
     public boolean scannerOperatorIsLeaf(IDataSource<AqlSourceId> dataSource) {
-        AqlSourceId asid = dataSource.getId();
-        String dataverseName = asid.getDataverseName();
-        String datasetName = asid.getDatasetName();
-        Dataset dataset = null;
-        try {
-            dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-        } catch (MetadataException e) {
-            throw new IllegalStateException(e);
+        boolean result = false;
+        switch (((AqlDataSource) dataSource).getDatasourceType()) {
+            case INTERNAL_DATASET:
+            case EXTERNAL_DATASET:
+                result = ((DatasetDataSource) dataSource).getDataset().getDatasetType() == DatasetType.EXTERNAL;
+                break;
+            case FEED:
+                result = true;
+                break;
         }
-
-        if (dataset == null) {
-            throw new IllegalArgumentException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
-        }
-        return dataset.getDatasetType() == DatasetType.EXTERNAL;
+        return result;
     }
 
     @Override
@@ -1415,8 +1392,8 @@
         try {
             File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
             Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-            if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
-                throw new AlgebricksException("Not an internal or feed dataset");
+            if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+                throw new AlgebricksException("Not an internal dataset");
             }
             InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
             List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
@@ -1512,6 +1489,14 @@
         return type.getDatatype();
     }
 
+    public Feed findFeed(String dataverse, String feedName) throws AlgebricksException {
+        try {
+            return MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, feedName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     public FeedPolicy findFeedPolicy(String dataverse, String policyName) throws AlgebricksException {
         try {
             return MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverse, policyName);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
new file mode 100644
index 0000000..23a6d84
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
@@ -0,0 +1,98 @@
+package edu.uci.ics.asterix.metadata.declared;
+
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.AsterixNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class DatasetDataSource extends AqlDataSource {
+
+    private Dataset dataset;
+
+    public DatasetDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
+            AqlDataSourceType datasourceType) throws AlgebricksException {
+        super(id, datasourceDataverse, datasourceName, itemType, datasourceType);
+        MetadataTransactionContext ctx = null;
+        try {
+            ctx = MetadataManager.INSTANCE.beginTransaction();
+            dataset = MetadataManager.INSTANCE.getDataset(ctx, datasourceDataverse, datasourceName);
+            if (dataset == null) {
+                throw new AlgebricksException("Unknown dataset " + datasourceName + " in dataverse "
+                        + datasourceDataverse);
+            }
+            MetadataManager.INSTANCE.commitTransaction(ctx);
+            switch (dataset.getDatasetType()) {
+                case INTERNAL:
+                    initInternalDataset(itemType);
+                    break;
+                case EXTERNAL:
+                    initExternalDataset(itemType);
+                    break;
+
+            }
+        } catch (Exception e) {
+            if (ctx != null) {
+                try {
+                    MetadataManager.INSTANCE.abortTransaction(ctx);
+                } catch (Exception e2) {
+                    e2.addSuppressed(e);
+                    throw new IllegalStateException("Unable to abort " + e2.getMessage());
+                }
+            }
+
+        }
+
+    }
+
+    public Dataset getDataset() {
+        return dataset;
+    }
+
+    private void initInternalDataset(IAType itemType) throws IOException {
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        ARecordType recordType = (ARecordType) itemType;
+        int n = partitioningKeys.size();
+        schemaTypes = new IAType[n + 1];
+        for (int i = 0; i < n; i++) {
+            schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
+        }
+        schemaTypes[n] = itemType;
+        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+    }
+
+    private void initExternalDataset(IAType itemType) {
+        schemaTypes = new IAType[1];
+        schemaTypes[0] = itemType;
+        INodeDomain domainForExternalData = new INodeDomain() {
+            @Override
+            public Integer cardinality() {
+                return null;
+            }
+
+            @Override
+            public boolean sameAs(INodeDomain domain) {
+                return domain == this;
+            }
+        };
+        domain = domainForExternalData;
+    }
+
+    @Override
+    public IAType[] getSchemaTypes() {
+        return schemaTypes;
+    }
+
+    @Override
+    public INodeDomain getDomain() {
+        return domain;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
deleted file mode 100644
index ed5df31..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public class ExternalFeedDataSource extends AqlDataSource {
-
-    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
-        super(id, dataset, itemType);
-    }
-
-    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType dataSourceType)
-            throws AlgebricksException {
-        super(id, dataset, itemType, dataSourceType);
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
new file mode 100644
index 0000000..fc5d9ca
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.declared;
+
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class FeedDataSource extends AqlDataSource {
+
+    private Feed feed;
+    private final FeedConnectionId feedConnectionId;
+
+    public FeedDataSource(AqlSourceId id, FeedConnectionId feedId, IAType itemType, AqlDataSourceType dataSourceType)
+            throws AlgebricksException {
+        super(id, feedId.getDataverse(), feedId.getFeedName(), itemType, dataSourceType);
+        this.feedConnectionId = feedId;
+        feed = null;
+        MetadataTransactionContext ctx = null;
+        try {
+            ctx = MetadataManager.INSTANCE.beginTransaction();
+            feed = MetadataManager.INSTANCE.getFeed(ctx, feedId.getDataverse(), feedId.getFeedName());
+            if (feed == null) {
+                throw new AlgebricksException("Unknown feed " + feedId);
+            }
+            MetadataManager.INSTANCE.commitTransaction(ctx);
+            initFeedDataSource(itemType);
+        } catch (Exception e) {
+            if (ctx != null) {
+                try {
+                    MetadataManager.INSTANCE.abortTransaction(ctx);
+                } catch (Exception e2) {
+                    e2.addSuppressed(e);
+                    throw new IllegalStateException("Unable to abort " + e2.getMessage());
+                }
+            }
+
+        }
+    }
+
+    public Feed getFeed() {
+        return feed;
+    }
+
+    @Override
+    public IAType[] getSchemaTypes() {
+        return schemaTypes;
+    }
+
+    @Override
+    public INodeDomain getDomain() {
+        return domain;
+    }
+
+    public FeedConnectionId getFeedConnectionId() {
+        return feedConnectionId;
+    }
+
+    private void initFeedDataSource(IAType itemType) {
+        schemaTypes = new IAType[1];
+        schemaTypes[0] = itemType;
+        INodeDomain domainForExternalData = new INodeDomain() {
+            @Override
+            public Integer cardinality() {
+                return null;
+            }
+
+            @Override
+            public boolean sameAs(INodeDomain domain) {
+                return domain == this;
+            }
+        };
+        domain = domainForExternalData;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index 7455cec..09db248 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -117,4 +117,9 @@
         }
         return true;
     }
+
+    @Override
+    public String toString() {
+        return dataverseName + "." + datasetName;
+    }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java
new file mode 100644
index 0000000..5d1704e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entities;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.IDatasetDetails;
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed.
+ */
+public class Feed implements IMetadataEntity {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String dataverseName;
+    private final String feedName;
+    private final String adaptorName;
+    private final Map<String, String> adaptorConfiguration;
+    private final FunctionSignature appliedFunction;
+
+    public Feed(String dataverseName, String datasetName, String adaptorName, Map<String, String> adaptorConfiguration,
+            FunctionSignature appliedFunction) {
+        this.dataverseName = dataverseName;
+        this.feedName = datasetName;
+        this.adaptorName = adaptorName;
+        this.adaptorConfiguration = adaptorConfiguration;
+        this.appliedFunction = appliedFunction;
+    }
+
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getFeedName() {
+        return feedName;
+    }
+
+    public String getAdaptorName() {
+        return adaptorName;
+    }
+
+    public Map<String, String> getAdaptorConfiguration() {
+        return adaptorConfiguration;
+    }
+
+    public FunctionSignature getAppliedFunction() {
+        return appliedFunction;
+    }
+
+    @Override
+    public Object addToCache(MetadataCache cache) {
+        return cache.addFeedIfNotExists(this);
+    }
+
+    @Override
+    public Object dropFromCache(MetadataCache cache) {
+        return cache.dropFeed(this);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof Feed)) {
+            return false;
+        }
+        Feed otherDataset = (Feed) other;
+        if (!otherDataset.dataverseName.equals(dataverseName)) {
+            return false;
+        }
+        if (!otherDataset.feedName.equals(feedName)) {
+            return false;
+        }
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
index 8e6dab3..c4f84e5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
@@ -30,8 +30,8 @@
     private int activityId;
 
     private final String dataverseName;
-    // Enforced to be unique within a dataverse.
     private final String datasetName;
+    private final String feedName;
 
     private String lastUpdatedTimestamp;
     private FeedActivityType activityType;
@@ -43,9 +43,7 @@
         FEED_FAILURE,
         FEED_RECOVERY,
         FEED_RESUME,
-        FEED_STATS,
-        FEED_EXPAND,
-        FEED_SHRINK
+        FEED_STATS
     }
 
     public static class FeedActivityDetails {
@@ -61,9 +59,10 @@
 
     }
 
-    public FeedActivity(String dataverseName, String datasetName, FeedActivityType feedActivityType,
+    public FeedActivity(String dataverseName, String feedName, String datasetName, FeedActivityType feedActivityType,
             Map<String, String> feedActivityDetails) {
         this.dataverseName = dataverseName;
+        this.feedName = feedName;
         this.datasetName = datasetName;
         this.activityType = feedActivityType;
         this.feedActivityDetails = feedActivityDetails;
@@ -77,6 +76,10 @@
         return datasetName;
     }
 
+    public String getFeedName() {
+        return feedName;
+    }
+
     @Override
     public Object addToCache(MetadataCache cache) {
         return cache.addFeedActivityIfNotExists(this);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
deleted file mode 100644
index e2268bf..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.entities;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.builders.IARecordBuilder;
-import edu.uci.ics.asterix.builders.OrderedListBuilder;
-import edu.uci.ics.asterix.builders.RecordBuilder;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
-import edu.uci.ics.asterix.om.base.AMutableString;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
-
-/**
- * Provides functionality for writing parameters for a FEED dataset into the
- * Metadata. Since FEED dataset is a special kind of INTERNAL dataset, this
- * class extends InternalDatasetDetails.
- */
-public class FeedDatasetDetails extends InternalDatasetDetails {
-
-    private static final long serialVersionUID = 1L;
-    private final String adapterFactory;
-    private final Map<String, String> properties;
-    private final FunctionSignature signature;
-
-    public enum FeedState {
-        // INACTIVE state signifies that the feed dataset is not
-        // connected with the external world through the feed
-        // adapter.
-        INACTIVE,
-
-        // ACTIVE state signifies that the feed dataset is connected to the
-        // external world using an adapter that may put data into the dataset.
-        ACTIVE
-    }
-
-    public FeedDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
-            List<String> partitioningKey, List<String> primaryKey, String groupName, String adapterFactory,
-            Map<String, String> properties, FunctionSignature signature) {
-        super(fileStructure, partitioningStrategy, partitioningKey, primaryKey, groupName);
-        this.properties = properties;
-        this.adapterFactory = adapterFactory;
-        this.signature = signature;
-    }
-
-    @Override
-    public DatasetType getDatasetType() {
-        return DatasetType.FEED;
-    }
-
-    @Override
-    public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
-        IARecordBuilder feedRecordBuilder = new RecordBuilder();
-        OrderedListBuilder listBuilder = new OrderedListBuilder();
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
-        feedRecordBuilder.reset(MetadataRecordTypes.FEED_DETAILS_RECORDTYPE);
-        AMutableString aString = new AMutableString("");
-        ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ASTRING);
-
-        // write field 0
-        fieldValue.reset();
-        aString.setValue(this.getFileStructure().toString());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX, fieldValue);
-
-        // write field 1
-        fieldValue.reset();
-        aString.setValue(this.getPartitioningStrategy().toString());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX, fieldValue);
-
-        // write field 2
-        listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[2]);
-        for (String field : partitioningKeys) {
-            itemValue.reset();
-            aString.setValue(field);
-            stringSerde.serialize(aString, itemValue.getDataOutput());
-            listBuilder.addItem(itemValue);
-        }
-        fieldValue.reset();
-        listBuilder.write(fieldValue.getDataOutput(), true);
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX, fieldValue);
-
-        // write field 3
-        listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[3]);
-        for (String field : primaryKeys) {
-            itemValue.reset();
-            aString.setValue(field);
-            stringSerde.serialize(aString, itemValue.getDataOutput());
-            listBuilder.addItem(itemValue);
-        }
-        fieldValue.reset();
-        listBuilder.write(fieldValue.getDataOutput(), true);
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX, fieldValue);
-
-        // write field 4
-        fieldValue.reset();
-        aString.setValue(getNodeGroupName());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
-
-        // write field 5
-        fieldValue.reset();
-        aString.setValue(getAdapterFactory());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX, fieldValue);
-
-        // write field 6
-        listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[6]);
-        for (Map.Entry<String, String> property : properties.entrySet()) {
-            String name = property.getKey();
-            String value = property.getValue();
-            itemValue.reset();
-            writePropertyTypeRecord(name, value, itemValue.getDataOutput());
-            listBuilder.addItem(itemValue);
-        }
-        fieldValue.reset();
-        listBuilder.write(fieldValue.getDataOutput(), true);
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
-
-        // write field 7
-        fieldValue.reset();
-        if (signature != null) {
-            aString.setValue(signature.toString());
-            stringSerde.serialize(aString, fieldValue.getDataOutput());
-            feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
-        }
-
-        try {
-            feedRecordBuilder.write(out, true);
-        } catch (IOException | AsterixException e) {
-            throw new HyracksDataException(e);
-        }
-
-    }
-
-    public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
-        IARecordBuilder propertyRecordBuilder = new RecordBuilder();
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        propertyRecordBuilder.reset(MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
-        AMutableString aString = new AMutableString("");
-        ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ASTRING);
-
-        // write field 0
-        fieldValue.reset();
-        aString.setValue(name);
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        propertyRecordBuilder.addField(0, fieldValue);
-
-        // write field 1
-        fieldValue.reset();
-        aString.setValue(value);
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        propertyRecordBuilder.addField(1, fieldValue);
-
-        try {
-            propertyRecordBuilder.write(out, true);
-        } catch (IOException | AsterixException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    public String getAdapterFactory() {
-        return adapterFactory;
-    }
-
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    public FunctionSignature getFunction() {
-        return signature;
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 9ba5848..f66ee7d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -31,7 +31,6 @@
 import edu.uci.ics.asterix.builders.UnorderedListBuilder;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.metadata.IDatasetDetails;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -39,14 +38,12 @@
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableInt32;
 import edu.uci.ics.asterix.om.base.AMutableString;
-import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.AOrderedList;
 import edu.uci.ics.asterix.om.base.ARecord;
 import edu.uci.ics.asterix.om.base.AString;
@@ -110,66 +107,7 @@
         int pendingOp = ((AInt32) datasetRecord
                 .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX)).getIntegerValue();
         switch (datasetType) {
-            case FEED: {
-                ARecord datasetDetailsRecord = (ARecord) datasetRecord
-                        .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX);
-                FileStructure fileStructure = FileStructure.valueOf(((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX))
-                        .getStringValue());
-                PartitioningStrategy partitioningStrategy = PartitioningStrategy
-                        .valueOf(((AString) datasetDetailsRecord
-                                .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX))
-                                .getStringValue());
-                IACursor cursor = ((AOrderedList) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX)).getCursor();
-                List<String> partitioningKey = new ArrayList<String>();
-                while (cursor.next())
-                    partitioningKey.add(((AString) cursor.get()).getStringValue());
-                String groupName = ((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX))
-                        .getStringValue();
-                String adapter = ((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX))
-                        .getStringValue();
-                cursor = ((AOrderedList) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX)).getCursor();
-                Map<String, String> properties = new HashMap<String, String>();
-                String key;
-                String value;
-                while (cursor.next()) {
-                    ARecord field = (ARecord) cursor.get();
-                    key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
-                            .getStringValue();
-                    value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
-                            .getStringValue();
-                    properties.put(key, value);
-                }
 
-                Object o = datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_FUNCTION_FIELD_INDEX);
-                FunctionSignature signature = null;
-                if (!(o instanceof ANull)) {
-                    String functionIdentifier = ((AString) o).getStringValue();
-                    String[] qnameComponents = functionIdentifier.split("\\.");
-                    String functionDataverse;
-                    String functionName;
-                    if (qnameComponents.length == 2) {
-                        functionDataverse = qnameComponents[0];
-                        functionName = qnameComponents[1];
-                    } else {
-                        functionDataverse = dataverseName;
-                        functionName = qnameComponents[0];
-                    }
-
-                    String[] nameComponents = functionName.split("@");
-                    signature = new FunctionSignature(functionDataverse, nameComponents[0],
-                            Integer.parseInt(nameComponents[1]));
-                }
-
-                datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
-                        partitioningKey, groupName, adapter, properties, signature);
-                break;
-            }
             case INTERNAL: {
                 ARecord datasetDetailsRecord = (ARecord) datasetRecord
                         .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX);
@@ -325,9 +263,6 @@
             case EXTERNAL:
                 recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX, fieldValue);
                 break;
-            case FEED:
-                recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX, fieldValue);
-                break;
         }
 
     }
@@ -341,10 +276,8 @@
         IACursor cursor = list.getCursor();
         while (cursor.next()) {
             ARecord field = (ARecord) cursor.get();
-            key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
-                    .getStringValue();
-            value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
-                    .getStringValue();
+            key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+            value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
             hints.put(key, value);
         }
         return hints;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
index 77240e1..8648f00 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
@@ -56,12 +56,14 @@
     // Key field.
     public static final int FEED_ACTIVITY_ACTIVITY_DATAVERSE_NAME_FIELD_INDEX = 0;
 
-    public static final int FEED_ACTIVITY_ACTIVITY_DATASET_NAME_FIELD_INDEX = 1;
+    public static final int FEED_ACTIVITY_ACTIVITY_FEED_NAME_FIELD_INDEX = 1;
 
-    public static final int FEED_ACTIVITY_ACTIVITY_ID_FIELD_INDEX = 2;
+    public static final int FEED_ACTIVITY_ACTIVITY_DATASET_NAME_FIELD_INDEX = 2;
+
+    public static final int FEED_ACTIVITY_ACTIVITY_ID_FIELD_INDEX = 3;
 
     // Payload field containing serialized FeedActivity.
-    public static final int FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+    public static final int FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX = 4;
 
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
@@ -91,6 +93,8 @@
 
         String dataverseName = ((AString) feedActivityRecord
                 .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+        String feedName = ((AString) feedActivityRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX)).getStringValue();
         String datasetName = ((AString) feedActivityRecord
                 .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX)).getStringValue();
         int activityId = ((AInt32) feedActivityRecord
@@ -110,8 +114,8 @@
             activityDetails.put(key, value);
         }
 
-        FeedActivity fa = new FeedActivity(dataverseName, datasetName, FeedActivityType.valueOf(feedActivityType),
-                activityDetails);
+        FeedActivity fa = new FeedActivity(dataverseName, feedName, datasetName,
+                FeedActivityType.valueOf(feedActivityType), activityDetails);
         fa.setActivityId(activityId);
         return fa;
     }
@@ -126,6 +130,10 @@
         stringSerde.serialize(aString, tupleBuilder.getDataOutput());
         tupleBuilder.addFieldEndOffset();
 
+        aString.setValue(feedActivity.getFeedName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
         aString.setValue(feedActivity.getDatasetName());
         stringSerde.serialize(aString, tupleBuilder.getDataOutput());
         tupleBuilder.addFieldEndOffset();
@@ -145,23 +153,29 @@
 
         // write field 1
         fieldValue.reset();
+        aString.setValue(feedActivity.getFeedName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 2
+        fieldValue.reset();
         aString.setValue(feedActivity.getDatasetName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX, fieldValue);
 
-        // write field 2
+        // write field 3
         fieldValue.reset();
         aInt32.setValue(feedActivity.getActivityId());
         int32Serde.serialize(aInt32, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX, fieldValue);
 
-        // write field 3
+        // write field 4
         fieldValue.reset();
         aString.setValue(feedActivity.getFeedActivityType().name());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX, fieldValue);
 
-        // write field 4
+        // write field 5
         Map<String, String> properties = feedActivity.getFeedActivityDetails();
         UnorderedListBuilder listBuilder = new UnorderedListBuilder();
         listBuilder
@@ -177,7 +191,7 @@
         listBuilder.write(fieldValue.getDataOutput(), true);
         recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX, fieldValue);
 
-        // write field 5
+        // write field 6
         fieldValue.reset();
         aString.setValue(Calendar.getInstance().getTime().toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
new file mode 100644
index 0000000..7de0118
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -0,0 +1,235 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Feed metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedTupleTranslator extends AbstractTupleTranslator<Feed> {
+    // Field indexes of serialized Feed in a tuple.
+    // Key field.
+    public static final int FEED_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+    public static final int FEED_NAME_FIELD_INDEX = 1;
+
+    // Payload field containing serialized feed.
+    public static final int FEED_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+    @SuppressWarnings("unchecked")
+    public FeedTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FEED_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1);
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+    }
+
+    @Override
+    public Feed getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+        byte[] serRecord = frameTuple.getFieldData(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordStartOffset = frameTuple.getFieldStart(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = frameTuple.getFieldLength(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord feedRecord = (ARecord) recordSerDes.deserialize(in);
+        return createFeedFromARecord(feedRecord);
+    }
+
+    private Feed createFeedFromARecord(ARecord feedRecord) {
+        Feed feed = null;
+        String dataverseName = ((AString) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+        String feedName = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX))
+                .getStringValue();
+        String adaptorName = ((AString) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX)).getStringValue();
+
+        IACursor cursor = ((AUnorderedList) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX)).getCursor();
+        String key;
+        String value;
+        Map<String, String> adaptorConfiguration = new HashMap<String, String>();
+        while (cursor.next()) {
+            ARecord field = (ARecord) cursor.get();
+            key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+            value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
+            adaptorConfiguration.put(key, value);
+        }
+
+        Object o = feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX);
+        FunctionSignature signature = null;
+        if (!(o instanceof ANull)) {
+            String functionIdentifier = ((AString) o).getStringValue();
+            String[] qnameComponents = functionIdentifier.split("\\.");
+            String functionDataverse;
+            String functionName;
+            if (qnameComponents.length == 2) {
+                functionDataverse = qnameComponents[0];
+                functionName = qnameComponents[1];
+            } else {
+                functionDataverse = dataverseName;
+                functionName = qnameComponents[0];
+            }
+
+            String[] nameComponents = functionName.split("@");
+            signature = new FunctionSignature(functionDataverse, nameComponents[0], Integer.parseInt(nameComponents[1]));
+        }
+
+        feed = new Feed(dataverseName, feedName, adaptorName, adaptorConfiguration, signature);
+        return feed;
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(Feed feed) throws IOException, MetadataException {
+        // write the key in the first three fields of the tuple
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+        tupleBuilder.reset();
+        aString.setValue(feed.getDataverseName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        aString.setValue(feed.getFeedName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        recordBuilder.reset(MetadataRecordTypes.FEED_RECORDTYPE);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(feed.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(feed.getFeedName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 2
+        fieldValue.reset();
+        aString.setValue(feed.getAdaptorName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 3 (adaptorConfiguration)
+        Map<String, String> adaptorConfiguration = feed.getAdaptorConfiguration();
+        UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+        listBuilder
+                .reset((AUnorderedListType) MetadataRecordTypes.FEED_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX]);
+        for (Map.Entry<String, String> property : adaptorConfiguration.entrySet()) {
+            String name = property.getKey();
+            String value = property.getValue();
+            itemValue.reset();
+            writePropertyTypeRecord(name, value, itemValue.getDataOutput());
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX, fieldValue);
+
+        // write field 4
+        fieldValue.reset();
+        if (feed.getAppliedFunction() != null) {
+            aString.setValue(feed.getAppliedFunction().toString());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
+        }
+
+        // write field 5
+        fieldValue.reset();
+        aString.setValue(Calendar.getInstance().getTime().toString());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
+
+        // write record
+        try {
+            recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        } catch (AsterixException e) {
+            throw new MetadataException(e);
+        }
+        tupleBuilder.addFieldEndOffset();
+
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+
+    public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
+        IARecordBuilder propertyRecordBuilder = new RecordBuilder();
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        propertyRecordBuilder.reset(MetadataRecordTypes.FEED_ADAPTOR_CONFIGURATION_RECORDTYPE);
+        AMutableString aString = new AMutableString("");
+        ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+                .getSerializerDeserializer(BuiltinType.ASTRING);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(name);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        propertyRecordBuilder.addField(0, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(value);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        propertyRecordBuilder.addField(1, fieldValue);
+
+        try {
+            propertyRecordBuilder.write(out, true);
+        } catch (IOException | AsterixException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
index 96dc8dc..fb62f3e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
@@ -30,7 +30,8 @@
         FeedPolicyAccessor policyAccessor = this.policyEnforcer.getFeedPolicyAccessor();
         if (policyAccessor.collectStatistics()) {
             long period = policyAccessor.getStatisicsCollectionPeriodInSecs();
-            collector = new StatisticsCollector(this, policyEnforcer.getFeedId(), period, batchPersistSize);
+            collector = new StatisticsCollector(this, policyEnforcer.getFeedId(), policyEnforcer.getFeedId()
+                    .getDatasetName(), period, batchPersistSize);
             timer = new Timer();
             timer.schedule(collector, period * 1000, period * 1000);
         }
@@ -45,7 +46,7 @@
     private static class StatisticsCollector extends TimerTask {
 
         private final AbstractFeedDatasourceAdapter adapter;
-        private final FeedId feedId;
+        private final FeedConnectionId feedId;
         //   private List<Long> previousCountValues = new ArrayList<Long>();
         //   private List<Integer> ingestionRates = new ArrayList<Integer>();
         private final long period;
@@ -56,13 +57,13 @@
         private StringBuilder rate = new StringBuilder();
         private long previousCount = 0;
 
-        public StatisticsCollector(AbstractFeedDatasourceAdapter adapter, FeedId feedId, long period,
-                int batchPersistSize) {
+        public StatisticsCollector(AbstractFeedDatasourceAdapter adapter, FeedConnectionId feedId,
+                String targetDataset, long period, int batchPersistSize) {
             this.adapter = adapter;
             this.feedId = feedId;
             this.period = period;
             this.batchPersistSize = batchPersistSize;
-            this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getDataset(),
+            this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getFeedName(), targetDataset,
                     FeedActivityType.FEED_STATS, new HashMap<String, String>());
         }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
index 5d9320c..62f779f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
@@ -27,7 +27,7 @@
 
     private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
 
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
 
     private IFeedAdapter feedAdapter;
 
@@ -56,7 +56,7 @@
         FINISHED_INGESTION
     }
 
-    public AdapterRuntimeManager(FeedId feedId, IFeedAdapter feedAdapter, MaterializingFrameWriter writer,
+    public AdapterRuntimeManager(FeedConnectionId feedId, IFeedAdapter feedAdapter, MaterializingFrameWriter writer,
             int partition, LinkedBlockingQueue<IFeedMessage> inbox) {
         this.feedId = feedId;
         this.feedAdapter = feedAdapter;
@@ -93,7 +93,7 @@
     }
 
     @Override
-    public FeedId getFeedId() {
+    public FeedConnectionId getFeedId() {
         return feedId;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
new file mode 100644
index 0000000..a487168
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.Serializable;
+
+/**
+ * A unique identifier for a data feed flowing into a dataset.
+ */
+public class FeedConnectionId implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String dataverse;
+    private final String feedName;
+    private final String datasetName;
+
+    public FeedConnectionId(String dataverse, String feedName, String datasetName) {
+        this.dataverse = dataverse;
+        this.feedName = feedName;
+        this.datasetName = datasetName;
+    }
+
+    public String getDataverse() {
+        return dataverse;
+    }
+
+    public String getFeedName() {
+        return feedName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || !(o instanceof FeedConnectionId)) {
+            return false;
+        }
+        if (((FeedConnectionId) o).getFeedName().equals(feedName)
+                && ((FeedConnectionId) o).getDataverse().equals(dataverse)
+                && ((FeedConnectionId) o).getDatasetName().equals(datasetName)) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return toString().hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return dataverse + "." + feedName + "-->" + datasetName;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedId.java
deleted file mode 100644
index d56081d..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedId.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.feeds;
-
-import java.io.Serializable;
-
-/**
- * A unique identifier for a feed (dataset).
- */
-public class FeedId implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    private final String dataverse;
-    private final String dataset;
-    private final int hashcode;
-
-    public FeedId(String dataverse, String dataset) {
-        this.dataset = dataset;
-        this.dataverse = dataverse;
-        this.hashcode = (dataverse + "." + dataset).hashCode();
-    }
-
-    public String getDataverse() {
-        return dataverse;
-    }
-
-    public String getDataset() {
-        return dataset;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == null || !(o instanceof FeedId)) {
-            return false;
-        }
-        if (((FeedId) o).getDataset().equals(dataset) && ((FeedId) o).getDataverse().equals(dataverse)) {
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public int hashCode() {
-        return hashcode;
-    }
-
-    @Override
-    public String toString() {
-        return dataverse + "." + dataset;
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
index d007d6f..ee87272 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -35,13 +35,13 @@
     private static final long serialVersionUID = 1L;
 
     private final IAType atype;
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
     private final Map<String, String> feedPolicy;
     private IAdapterFactory adapterFactory;
 
-    public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedId feedId, IAdapterFactory adapterFactory,
+    public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, IAdapterFactory adapterFactory,
             ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicy) {
-        super(spec, 1, 1);
+        super(spec, 0, 1);
         recordDescriptors[0] = rDesc;
         this.adapterFactory = adapterFactory;
         this.atype = atype;
@@ -61,7 +61,7 @@
         return new FeedIntakeOperatorNodePushable(feedId, adapter, feedPolicy, partition);
     }
 
-    public FeedId getFeedId() {
+    public FeedConnectionId getFeedId() {
         return feedId;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
index 0d46e43..d6b7178 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
-import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.logging.Level;
@@ -22,25 +21,25 @@
 
 import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager.State;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
 /**
  * The runtime for @see{FeedIntakeOperationDescriptor}
  */
-public class FeedIntakeOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
 
     private static Logger LOGGER = Logger.getLogger(FeedIntakeOperatorNodePushable.class.getName());
 
     private IFeedAdapter adapter;
     private final int partition;
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
     private final LinkedBlockingQueue<IFeedMessage> inbox;
     private final Map<String, String> feedPolicy;
     private final FeedPolicyEnforcer policyEnforcer;
     private AdapterRuntimeManager adapterRuntimeMgr;
 
-    public FeedIntakeOperatorNodePushable(FeedId feedId, IFeedAdapter adapter, Map<String, String> feedPolicy,
-            int partition) {
+    public FeedIntakeOperatorNodePushable(FeedConnectionId feedId, IFeedAdapter adapter,
+            Map<String, String> feedPolicy, int partition) {
         this.adapter = adapter;
         this.partition = partition;
         this.feedId = feedId;
@@ -50,7 +49,7 @@
     }
 
     @Override
-    public void open() throws HyracksDataException {
+    public void initialize() throws HyracksDataException {
         adapterRuntimeMgr = FeedManager.INSTANCE.getFeedRuntimeManager(feedId, partition);
         try {
             if (adapterRuntimeMgr == null) {
@@ -97,21 +96,6 @@
         }
     }
 
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.close();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        // do nothing
-    }
-
     public Map<String, String> getFeedPolicy() {
         return feedPolicy;
     }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
index 718310b..ce58efb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
@@ -30,7 +30,7 @@
 
     }
 
-    private Map<FeedId, List<AdapterRuntimeManager>> activeFeedRuntimeManagers = new HashMap<FeedId, List<AdapterRuntimeManager>>();
+    private Map<FeedConnectionId, List<AdapterRuntimeManager>> activeFeedRuntimeManagers = new HashMap<FeedConnectionId, List<AdapterRuntimeManager>>();
 
     @Override
     public synchronized void registerFeedRuntime(AdapterRuntimeManager adapterRuntimeMgr) {
@@ -51,7 +51,7 @@
     }
 
     @Override
-    public synchronized AdapterRuntimeManager getFeedRuntimeManager(FeedId feedId, int partition) {
+    public synchronized AdapterRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId, int partition) {
         List<AdapterRuntimeManager> adapterRuntimeMgrs = activeFeedRuntimeManagers.get(feedId);
         if (adapterRuntimeMgrs != null) {
             for (AdapterRuntimeManager mgr : adapterRuntimeMgrs) {
@@ -63,7 +63,7 @@
         return null;
     }
 
-    public List<AdapterRuntimeManager> getFeedRuntimeManagers(FeedId feedId) {
+    public List<AdapterRuntimeManager> getFeedRuntimeManagers(FeedConnectionId feedId) {
         return activeFeedRuntimeManagers.get(feedId);
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
index a302978..b726955 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
@@ -28,13 +28,13 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
     private final IFeedMessage feedMessage;
 
-    public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String dataset,
+    public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String feedName, String dataset,
             IFeedMessage feedMessage) {
         super(spec, 0, 1);
-        this.feedId = new FeedId(dataverse, dataset);
+        this.feedId = new FeedConnectionId(dataverse, feedName, dataset);
         this.feedMessage = feedMessage;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
index 4ffa8a9..5c41675 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
@@ -29,11 +29,11 @@
 
     private static final Logger LOGGER = Logger.getLogger(FeedMessageOperatorNodePushable.class.getName());
 
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
     private final IFeedMessage feedMessage;
     private final int partition;
 
-    public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IFeedMessage feedMessage,
+    public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedMessage feedMessage,
             int partition, int nPartitions) {
         this.feedId = feedId;
         this.feedMessage = feedMessage;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
index 0f8bcee..b94e228 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
@@ -13,15 +13,15 @@
 
 public class FeedPolicyEnforcer {
 
-    private final FeedId feedId;
+    private final FeedConnectionId feedId;
     private final FeedPolicyAccessor feedPolicyAccessor;
     private final FeedActivity feedActivity;
 
-    public FeedPolicyEnforcer(FeedId feedId, Map<String, String> feedPolicy) {
+    public FeedPolicyEnforcer(FeedConnectionId feedId, Map<String, String> feedPolicy) {
         this.feedId = feedId;
         this.feedPolicyAccessor = new FeedPolicyAccessor(feedPolicy);
-        this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getDataset(), null,
-                new HashMap<String, String>());
+        this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getFeedName(), feedId.getDatasetName(),
+                null, new HashMap<String, String>());
     }
 
     public boolean handleSoftwareFailure(Exception e) throws RemoteException, ACIDException {
@@ -57,7 +57,7 @@
         return feedPolicyAccessor;
     }
 
-    public FeedId getFeedId() {
+    public FeedConnectionId getFeedId() {
         return feedId;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
index 47ccaf1..c8a011d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
@@ -6,7 +6,7 @@
 public class FeedUtil {
 
     public static boolean isFeedActive(FeedActivity feedActivity) {
-        return (feedActivity != null && (!feedActivity.getActivityType().equals(FeedActivityType.FEED_END) && !feedActivity
+        return (feedActivity != null && !(feedActivity.getActivityType().equals(FeedActivityType.FEED_END) || feedActivity
                 .getActivityType().equals(FeedActivityType.FEED_FAILURE)));
     }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
index 0f71166..d6f6f55 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
@@ -6,6 +6,6 @@
 
     public void stop() throws Exception;
 
-    public FeedId getFeedId();
+    public FeedConnectionId getFeedId();
 
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
index 46302e8..c1147af 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
@@ -34,6 +34,6 @@
      * @param partition
      * @return
      */
-    public AdapterRuntimeManager getFeedRuntimeManager(FeedId feedId, int partition);
+    public AdapterRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId, int partition);
 
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index 639b77a..5ab32b5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -95,43 +95,38 @@
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
                     IMetadataProvider<?, ?> mp) throws AlgebricksException {
                 AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
-                if (f.getArguments().size() != 1) {
-                    throw new AlgebricksException("dataset arity is 1, not " + f.getArguments().size());
+                if (f.getArguments().size() != 3) {
+                    throw new AlgebricksException("Incorrect number of arguments -> arity is 3, not "
+                            + f.getArguments().size());
                 }
-                ILogicalExpression a1 = f.getArguments().get(0).getValue();
+                ILogicalExpression a1 = f.getArguments().get(1).getValue();
                 IAType t1 = (IAType) env.getType(a1);
                 if (t1.getTypeTag() == ATypeTag.ANY) {
                     return BuiltinType.ANY;
                 }
                 if (t1.getTypeTag() != ATypeTag.STRING) {
-                    throw new AlgebricksException("Illegal type " + t1 + " for dataset() argument.");
+                    throw new AlgebricksException("Illegal type " + t1 + " for feed-ingest argument.");
                 }
                 if (a1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
                     return BuiltinType.ANY;
                 }
                 AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
-                String datasetArg = ((AString) acv.getObject()).getStringValue();
+                String typeArg = ((AString) acv.getObject()).getStringValue();
                 AqlMetadataProvider metadata = ((AqlMetadataProvider) mp);
-                Pair<String, String> datasetInfo = getDatasetInfo(metadata, datasetArg);
-                String dataverseName = datasetInfo.first;
-                String datasetName = datasetInfo.second;
+                Pair<String, String> argInfo = getDatasetInfo(metadata, typeArg);
+                String dataverseName = argInfo.first;
+                String typeName = argInfo.second;
                 if (dataverseName == null) {
                     throw new AlgebricksException("Unspecified dataverse!");
                 }
-                Dataset dataset = metadata.findDataset(dataverseName, datasetName);
-                if (dataset == null) {
-                    throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse "
-                            + dataverseName);
-                }
-                String tn = dataset.getItemTypeName();
-                IAType t2 = metadata.findType(dataverseName, tn);
+                IAType t2 = metadata.findType(dataverseName, typeName);
                 if (t2 == null) {
-                    throw new AlgebricksException("No type for dataset " + datasetName);
+                    throw new AlgebricksException("Unknown type  " + typeName);
                 }
                 return t2;
             }
         });
-        
+
         AsterixBuiltinFunctions.addFunction(AsterixBuiltinFunctions.FEED_INTERCEPT, new IResultTypeComputer() {
 
             @Override
@@ -177,17 +172,16 @@
     }
 
     private static Pair<String, String> getDatasetInfo(AqlMetadataProvider metadata, String datasetArg) {
-        String[] datasetNameComponents = datasetArg.split("\\.");
-        String dataverseName;
-        String datasetName;
-        if (datasetNameComponents.length == 1) {
-            dataverseName = metadata.getDefaultDataverse() == null ? null : metadata.getDefaultDataverse()
-                    .getDataverseName();
-            datasetName = datasetNameComponents[0];
+        String[] nameComponents = datasetArg.split("\\.");
+        String first;
+        String second;
+        if (nameComponents.length == 1) {
+            first = metadata.getDefaultDataverse() == null ? null : metadata.getDefaultDataverse().getDataverseName();
+            second = nameComponents[0];
         } else {
-            dataverseName = datasetNameComponents[0];
-            datasetName = datasetNameComponents[1];
+            first = nameComponents[0];
+            second = nameComponents[1];
         }
-        return new Pair(dataverseName, datasetName);
+        return new Pair(first, second);
     }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index e3be0df..42acae6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -241,7 +241,7 @@
 
     public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
     public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "feed-ingest", 1);
+            "feed-ingest", 3);
     public final static FunctionIdentifier FEED_INTERCEPT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "feed-intercept", 1);
 
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
index 408d648..11cbe54 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
@@ -23,7 +23,7 @@
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
 import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
@@ -57,7 +57,7 @@
 
     @Override
     public String getName() {
-        return "pull_twitter_feed";
+        return "synthetic_twitter_feed";
     }
 
     @Override
@@ -86,7 +86,7 @@
         try {
             ctx = MetadataManager.INSTANCE.beginTransaction();
             Dataset ds = MetadataManager.INSTANCE.getDataset(ctx, dataverse, dataset);
-            String nodegroupName = ((FeedDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
+            String nodegroupName = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
             ng = MetadataManager.INSTANCE.getNodegroup(ctx, nodegroupName);
             MetadataManager.INSTANCE.commitTransaction(ctx);
         } catch (Exception e) {
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
index 137824a..b0ff634 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
@@ -26,7 +26,7 @@
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
 import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
@@ -101,7 +101,7 @@
                 MetadataManager.INSTANCE.acquireReadLatch();
                 ctx = MetadataManager.INSTANCE.beginTransaction();
                 Dataset ds = MetadataManager.INSTANCE.getDataset(ctx, dataverse, dataset);
-                String nodegroupName = ((FeedDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
+                String nodegroupName = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
                 ng = MetadataManager.INSTANCE.getNodegroup(ctx, nodegroupName);
                 MetadataManager.INSTANCE.commitTransaction(ctx);
             } catch (Exception e) {