checkpoint
diff --git a/.gitignore b/.gitignore
index 3707fd7..af99107 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,6 +3,12 @@
 .settings
 .project
 ClusterControllerService
+rttest
+mdtest
+ittest
+asterix_logs
+build
+bin
 asterix-app/rttest
 asterix-app/mdtest/
 asterix-app/opttest/
diff --git a/asterix-algebra/pom.xml b/asterix-algebra/pom.xml
index 074d693..09db13b 100644
--- a/asterix-algebra/pom.xml
+++ b/asterix-algebra/pom.xml
@@ -106,6 +106,12 @@
 			<version>0.8.1-SNAPSHOT</version>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-external-data</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
                 <dependency>
                         <groupId>edu.uci.ics.asterix</groupId>
                         <artifactId>asterix-transactions</artifactId>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
index 6a48a17..7fc9bcf 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
@@ -18,10 +18,13 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionDescriptorTag;
+import edu.uci.ics.asterix.external.library.ExternalFunctionDescriptorProvider;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.comparisons.ComparisonEvalFactory;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
@@ -138,9 +141,18 @@
         }
 
         IFunctionDescriptor fd = null;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        IDataFormat format = FormatUtils.getDefaultFormat();
-        fd = format.resolveFunction(expr, env);
+        if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
+            AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+ IDataFormat format = FormatUtils.getDefaultFormat();
+            fd = format.resolveFunction(expr, env);
+        } else {
+            try {
+                fd = ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor((IExternalFunctionInfo) expr
+                        .getFunctionInfo());
+            } catch (AsterixException ae) {
+                throw new AlgebricksException(ae);
+            }
+        }
         return fd.createEvaluatorFactory(args);
     }
 
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 ffde764..6b4cdb1 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
@@ -48,6 +48,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;
@@ -244,6 +245,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..301c685 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
@@ -25,6 +25,8 @@
 import edu.uci.ics.asterix.algebra.base.AsterixOperatorAnnotations;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -129,11 +131,12 @@
         }
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
+
         Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
         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();
@@ -302,14 +305,16 @@
                         ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
                         if (e1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
                             IDataSource<AqlSourceId> dataSource = (IDataSource<AqlSourceId>) scan.getDataSource();
+                            if (((AqlDataSource) dataSource).getDatasourceType().equals(AqlDataSourceType.FEED)) {
+                                return false;
+                            }
                             AqlSourceId asid = dataSource.getId();
                             AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                             Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
                             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 642ef22..c087bd1 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,13 +23,20 @@
 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.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -104,14 +111,13 @@
 
                 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());
                     }
                 }
                 v.add(unnest.getVariable());
-
                 DataSourceScanOperator scan = new DataSourceScanOperator(v, metadataProvider.findDataSource(asid));
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
@@ -126,40 +132,36 @@
                 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, feedName);
+                String policyName = metadataProvider.getConfig().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+                FeedPolicy policy = metadataProvider.findFeedPolicy(dataverseName, policyName);
+                if (policy == null) {
+                    policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
+                    if (policy == null) {
+                        throw new AlgebricksException("Unknown feed policy:" + policyName);
+                    }
                 }
 
-                AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
                 v.add(unnest.getVariable());
 
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
-                        metadataProvider));
+                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());
@@ -169,6 +171,7 @@
 
                 return true;
             }
+
         }
 
         return false;
@@ -183,18 +186,18 @@
         context.addPrimaryKey(pk);
     }
 
-    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
-            AqlMetadataProvider metadataProvider) 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);
-        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)
@@ -215,4 +218,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/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index c67a4e5..d25dba3 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -223,7 +223,8 @@
         boolean changed = false;
         for (int j = 0; j < args.size(); j++) {
             ILogicalExpression arg = args.get(j).getValue();
-            IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY) ? (IAType) env.getType(arg) : inputItemType;
+            IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY) ? (IAType) env
+                    .getType(arg) : inputItemType;
             switch (arg.getExpressionTag()) {
                 case FUNCTION_CALL:
                     ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
@@ -252,6 +253,10 @@
      */
     private static boolean staticRecordTypeCast(AbstractFunctionCallExpression func, ARecordType reqType,
             ARecordType inputType, IVariableTypeEnvironment env) throws AlgebricksException {
+        if (!(func.getFunctionIdentifier() == AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR || func
+                .getFunctionIdentifier() == AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR)) {
+            return false;
+        }
         IAType[] reqFieldTypes = reqType.getFieldTypes();
         String[] reqFieldNames = reqType.getFieldNames();
         IAType[] inputFieldTypes = inputType.getFieldTypes();
@@ -463,12 +468,9 @@
                     reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
                     fi = AsterixBuiltinFunctions.CAST_LIST;
             }
-            if (fi != null
-                    && ! inputFieldType.equals(reqFieldType)
-                    && parameterVars.size() > 0) {
+            if (fi != null && !inputFieldType.equals(reqFieldType) && parameterVars.size() > 0) {
                 //inject dynamic type casting
-                injectCastFunction(FunctionUtils.getFunctionInfo(fi),
-                        reqFieldType, inputFieldType, expRef, argExpr);
+                injectCastFunction(FunctionUtils.getFunctionInfo(fi), reqFieldType, inputFieldType, expRef, argExpr);
                 castInjected = true;
             }
             if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
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 7b12550..31095f9 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,20 +26,22 @@
 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.CompactStatement;
-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.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;
@@ -93,13 +95,15 @@
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 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.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.functions.ExternalFunctionCompilerUtil;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
@@ -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;
     }
 
@@ -509,13 +513,18 @@
             return null;
         }
         AbstractFunctionCallExpression f = null;
-        if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+        if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
+            IFunctionInfo finfo = ExternalFunctionCompilerUtil.getExternalFunctionInfo(
+                    metadataProvider.getMetadataTxnContext(), function);
+            f = new ScalarFunctionCallExpression(finfo, args);
+        } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
             IFunctionInfo finfo = new AsterixFunctionInfo(signature);
-            return new ScalarFunctionCallExpression(finfo, args);
+            f = new ScalarFunctionCallExpression(finfo, args);
         } else {
             throw new MetadataException(" User defined functions written in " + function.getLanguage()
                     + " are not supported");
         }
+        return f;
     }
 
     private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
@@ -1270,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;
@@ -1426,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 4d11dbd..5c4126f 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,11 +26,12 @@
 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.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -40,6 +41,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;
@@ -1423,7 +1425,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;
@@ -1444,7 +1446,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 d9aa047..486aeee 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;
@@ -321,15 +320,20 @@
         }
     }
 
-    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, 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;
             this.varCounter = varCounter;
         }
@@ -339,6 +343,10 @@
             return dataverseName;
         }
 
+        public String getFeedName() {
+            return feedName;
+        }
+
         @Override
         public String getDatasetName() {
             return datasetName;
@@ -358,24 +366,25 @@
 
         @Override
         public Kind getKind() {
-            return Kind.BEGIN_FEED;
+            return Kind.CONNECT_FEED;
+        }
+
+        public String getPolicyName() {
+            return policyName;
         }
     }
 
-    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
@@ -388,8 +397,8 @@
             return datasetName;
         }
 
-        public OperationType getOperationType() {
-            return operationType;
+        public String getFeedName() {
+            return feedName;
         }
 
         public int getVarCounter() {
@@ -402,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 {
diff --git a/asterix-app/.gitignore b/asterix-app/.gitignore
deleted file mode 100644
index ea8c4bf..0000000
--- a/asterix-app/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index e80e33d..06d289e 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -90,7 +86,7 @@
 						<additionalClasspathElement>${basedir}/src/main/resources</additionalClasspathElement> 
 						</additionalClasspathElements> -->
 					<forkMode>pertest</forkMode>
-					<argLine>-enableassertions -Xmx${test.heap.size}m 
+					<argLine>-enableassertions -Xmx${test.heap.size}m
 						-Dfile.encoding=UTF-8
 						-Djava.util.logging.config.file=src/test/resources/logging.properties
 						-Xdebug
@@ -106,12 +102,6 @@
 
 	<dependencies>
 		<dependency>
-			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-algebra</artifactId>
-			<version>0.8.1-SNAPSHOT</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
 			<groupId>javax.servlet</groupId>
 			<artifactId>servlet-api</artifactId>
 			<type>jar</type>
@@ -149,9 +139,15 @@
 			<artifactId>algebricks-compiler</artifactId>
 		</dependency>
 		<dependency>
-            <groupId>edu.uci.ics.hyracks</groupId>
-            <artifactId>hyracks-client</artifactId>
-        </dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-client</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-algebra</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
 			<artifactId>asterix-aql</artifactId>
@@ -187,19 +183,22 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
-                <dependency>
-                        <groupId>edu.uci.ics.asterix</groupId>
-                        <artifactId>asterix-common</artifactId>
-                        <version>0.8.1-SNAPSHOT</version>
-                        <type>test-jar</type>
-                        <scope>test</scope>
-                </dependency>
-                <dependency>
-                        <groupId>edu.uci.ics.asterix</groupId>
-                        <artifactId>asterix-transactions</artifactId>
-                        <version>0.8.1-SNAPSHOT</version>
-                        <scope>compile</scope>
-                </dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-common</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+		<!-- posssible remove this <dependency> <groupId>com.kenai.nbpwr</groupId> 
+			<artifactId>org-apache-commons-io</artifactId> <version>1.3.1-201002241208</version> 
+			<scope>test</scope> </dependency> -->
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-transactions</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
@@ -227,12 +226,12 @@
 			<type>jar</type>
 			<scope>test</scope>
 		</dependency>
-                <dependency>
-                	<groupId>edu.uci.ics.asterix</groupId>
-                	<artifactId>asterix-test-framework</artifactId>
-                	<version>0.8.1-SNAPSHOT</version>
-                	<scope>test</scope>
-                </dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-test-framework</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-app/scripts/asterix/startallncs.sh b/asterix-app/scripts/asterix/startallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/startcc.sh b/asterix-app/scripts/asterix/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/startnc.sh b/asterix-app/scripts/asterix/startnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopallncs.sh b/asterix-app/scripts/asterix/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopcc.sh b/asterix-app/scripts/asterix/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopnc.sh b/asterix-app/scripts/asterix/stopnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/test.properties b/asterix-app/scripts/asterix/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startcc.sh b/asterix-app/scripts/idefix/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startnc1.sh b/asterix-app/scripts/idefix/startnc1.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startnc2.sh b/asterix-app/scripts/idefix/startnc2.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/stopallncs.sh b/asterix-app/scripts/idefix/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/stopcc.sh b/asterix-app/scripts/idefix/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/execute.sh b/asterix-app/scripts/rainbow/execute.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startallncs.sh b/asterix-app/scripts/rainbow/startallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startcc.sh b/asterix-app/scripts/rainbow/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startnc.sh b/asterix-app/scripts/rainbow/startnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopallncs.sh b/asterix-app/scripts/rainbow/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopcc.sh b/asterix-app/scripts/rainbow/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopnc.sh b/asterix-app/scripts/rainbow/stopnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index 16e3c14..c2c75bc 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -64,6 +64,9 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
 
 public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
+
+    public static final AsterixPropertiesAccessor ASTERIX_PROPERTIES_ACCESSOR = createAsterixPropertiesAccessor();
+
     private static final int METADATA_IO_DEVICE_ID = 0;
 
     private ILSMMergePolicyFactory metadataMergePolicyFactory;
@@ -86,18 +89,26 @@
     private IIOManager ioManager;
     private boolean isShuttingdown;
 
-    public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
+    public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) throws AsterixException {
         this.ncApplicationContext = ncApplicationContext;
+        compilerProperties = new AsterixCompilerProperties(ASTERIX_PROPERTIES_ACCESSOR);
+        externalProperties = new AsterixExternalProperties(ASTERIX_PROPERTIES_ACCESSOR);
+        metadataProperties = new AsterixMetadataProperties(ASTERIX_PROPERTIES_ACCESSOR);
+        storageProperties = new AsterixStorageProperties(ASTERIX_PROPERTIES_ACCESSOR);
+        txnProperties = new AsterixTransactionProperties(ASTERIX_PROPERTIES_ACCESSOR);
+    }
+
+    private static AsterixPropertiesAccessor createAsterixPropertiesAccessor() {
+        AsterixPropertiesAccessor propertiesAccessor = null;
+        try {
+            propertiesAccessor = new AsterixPropertiesAccessor();
+        } catch (AsterixException e) {
+            throw new IllegalStateException("Unable to create properties accessor");
+        }
+        return propertiesAccessor;
     }
 
     public void initialize() throws IOException, ACIDException, AsterixException {
-        AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
-        compilerProperties = new AsterixCompilerProperties(propertiesAccessor);
-        externalProperties = new AsterixExternalProperties(propertiesAccessor);
-        metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
-        storageProperties = new AsterixStorageProperties(propertiesAccessor);
-        txnProperties = new AsterixTransactionProperties(propertiesAccessor);
-
         Logger.getLogger("edu.uci.ics").setLevel(externalProperties.getLogLevel());
 
         fileMapManager = new AsterixFileMapManager();
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/FeedDashboardServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java
new file mode 100644
index 0000000..f044ac5
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java
@@ -0,0 +1,133 @@
+/*
+ * 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.api.http.servlet;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.imageio.ImageIO;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+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.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.feeds.FeedConnectionId;
+
+public class FeedDashboardServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(FeedDashboardServlet.class.getName());
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        String resourcePath = null;
+        String requestURI = request.getRequestURI();
+
+        if (requestURI.equals("/")) {
+            response.setContentType("text/html");
+            resourcePath = "/feed/dashboard.html";
+        } else {
+            resourcePath = requestURI + ".html";
+        }
+
+        try {
+            InputStream is = FeedDashboardServlet.class.getResourceAsStream(resourcePath);
+            if (is == null) {
+                response.sendError(HttpServletResponse.SC_NOT_FOUND);
+                return;
+            }
+
+            // Special handler for font files and .png resources
+            if (resourcePath.endsWith(".png")) {
+
+                BufferedImage img = ImageIO.read(is);
+                OutputStream outputStream = response.getOutputStream();
+                String formatName = "png";
+                response.setContentType("image/png");
+                ImageIO.write(img, formatName, outputStream);
+                outputStream.close();
+                return;
+
+            }
+
+            response.setCharacterEncoding("utf-8");
+            InputStreamReader isr = new InputStreamReader(is);
+            StringBuilder sb = new StringBuilder();
+            BufferedReader br = new BufferedReader(isr);
+            String line = br.readLine();
+
+            while (line != null) {
+                sb.append(line + "\n");
+                line = br.readLine();
+            }
+
+            String feedName = request.getParameter("feed");
+            String datasetName = request.getParameter("dataset");
+            String dataverseName = request.getParameter("dataverse");
+
+            FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+
+            String outStr = null;
+            if (requestURI.startsWith("/webui/static")) {
+                outStr = sb.toString();
+            } else {
+                MetadataManager.INSTANCE.init();
+                MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+                FeedActivity activity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(ctx, feedId,
+                        FeedActivityType.FEED_BEGIN);
+                MetadataManager.INSTANCE.commitTransaction(ctx);
+
+                Map<String, String> activityDetails = activity.getFeedActivityDetails();
+
+                String host = activityDetails.get(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_HOST);
+                int port = Integer.parseInt(activityDetails
+                        .get(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_PORT));
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(" Super Feed Maanger address :" + host + "[" + port + "]");
+                }
+
+                String ingestLocations = activityDetails.get(FeedActivityDetails.INGEST_LOCATIONS);
+                String computeLocations = activityDetails.get(FeedActivityDetails.COMPUTE_LOCATIONS);
+                String storageLocations = activityDetails.get(FeedActivityDetails.STORAGE_LOCATIONS);
+                String ingestionPolicy = activityDetails.get(FeedActivityDetails.FEED_POLICY_NAME);
+                String activeSince = activity.getLastUpdatedTimestamp();
+
+                outStr = String.format(sb.toString(), dataverseName, datasetName, feedName, ingestLocations,
+                        computeLocations, storageLocations, ingestionPolicy, activeSince);
+                FeedServletUtil.initiateSubscription(feedId, host, port);
+            }
+
+            PrintWriter out = response.getWriter();
+            out.println(outStr);
+        } catch (ACIDException | MetadataException e) {
+            e.printStackTrace();
+        }
+    }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java
new file mode 100644
index 0000000..6985019
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java
@@ -0,0 +1,119 @@
+/*
+ * 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.api.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+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.feeds.FeedConnectionId;
+
+public class FeedDataProviderServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+
+        String feedName = request.getParameter("feed");
+        String datasetName = request.getParameter("dataset");
+        String dataverseName = request.getParameter("dataverse");
+
+        String report = getFeedReport(feedName, datasetName, dataverseName);
+        System.out.println(" REPORT " + report);
+        long timestamp = System.currentTimeMillis();
+        JSONObject obj = null;
+        if (report != null) {
+            JSONArray array = new JSONArray();
+            try {
+                obj = new JSONObject();
+                obj.put("type", "report");
+                obj.put("time", timestamp);
+                obj.put("value", report);
+            } catch (JSONException jsoe) {
+                throw new IOException(jsoe);
+            }
+        } else {
+            obj = verifyIfFeedIsAlive(dataverseName, feedName, datasetName);
+        }
+
+        PrintWriter out = response.getWriter();
+        out.println(obj.toString());
+    }
+
+    private String getFeedReport(String feedName, String datasetName, String dataverseName) {
+        FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+        LinkedBlockingQueue<String> queue = FeedLifecycleListener.INSTANCE.getFeedReportQueue(feedId);
+        String report = null;
+        try {
+            report = queue.poll(25, TimeUnit.SECONDS);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        return report;
+    }
+
+    private JSONObject verifyIfFeedIsAlive(String dataverseName, String feedName, String datasetName) {
+        JSONObject obj = new JSONObject();
+        try {
+            MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+            List<FeedActivity> feedActivities = MetadataManager.INSTANCE
+                    .getActiveFeeds(ctx, dataverseName, datasetName);
+            FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+            FeedActivity activity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(ctx, feedId, null);
+            switch(activity.getActivityType()){
+                case FEED_BEGIN:
+                    Map<String, String> activityDetails = activity.getFeedActivityDetails();
+                    String ingestLocations = activityDetails.get(FeedActivityDetails.INGEST_LOCATIONS);
+                    String computeLocations = activityDetails.get(FeedActivityDetails.COMPUTE_LOCATIONS);
+                    String storageLocations = activityDetails.get(FeedActivityDetails.STORAGE_LOCATIONS);
+                    obj.put("status", "active");
+                    obj.put("type", "reload");
+                    obj.put("ingestLocations", ingestLocations);
+                    obj.put("computeLocations", computeLocations);
+                    obj.put("storageLocations", storageLocations);
+                    System.out.println(" RE LOADING " + " ingestion at " + ingestLocations + " compute at "
+                            + computeLocations + " storage at " + storageLocations);
+                    break;
+                case FEED_FAILURE:
+                    obj.put("status", "failed");
+                    break;
+                case FEED_END:
+                    obj.put("status", "ended");
+                    break;
+            }
+        } catch (Exception e) {
+            // ignore
+        }
+        return obj;
+
+    }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java
new file mode 100644
index 0000000..546ff4f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java
@@ -0,0 +1,123 @@
+/*
+ * 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.api.http.servlet;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.List;
+
+import javax.imageio.ImageIO;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+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.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+
+public class FeedServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        String resourcePath = null;
+        String requestURI = request.getRequestURI();
+
+        if (requestURI.equals("/")) {
+            response.setContentType("text/html");
+            resourcePath = "/feed/home.html";
+        } else {
+            resourcePath = requestURI;
+        }
+
+        try {
+            InputStream is = FeedServlet.class.getResourceAsStream(resourcePath);
+            if (is == null) {
+                response.sendError(HttpServletResponse.SC_NOT_FOUND);
+                return;
+            }
+
+            // Special handler for font files and .png resources
+            if (resourcePath.endsWith(".png")) {
+
+                BufferedImage img = ImageIO.read(is);
+                OutputStream outputStream = response.getOutputStream();
+                String formatName = "png";
+                response.setContentType("image/png");
+                ImageIO.write(img, formatName, outputStream);
+                outputStream.close();
+                return;
+
+            }
+
+            response.setCharacterEncoding("utf-8");
+            InputStreamReader isr = new InputStreamReader(is);
+            StringBuilder sb = new StringBuilder();
+            BufferedReader br = new BufferedReader(isr);
+            String line = br.readLine();
+
+            while (line != null) {
+                sb.append(line + "\n");
+                line = br.readLine();
+            }
+
+            String outStr = null;
+            if (requestURI.startsWith("/webui/static")) {
+                outStr = sb.toString();
+            } else {
+                MetadataManager.INSTANCE.init();
+                MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+                List<FeedActivity> lfa = MetadataManager.INSTANCE.getActiveFeeds(ctx, null, null);
+                StringBuilder ldStr = new StringBuilder();
+                ldStr.append("<br />");
+                ldStr.append("<br />");
+                if (lfa == null || lfa.isEmpty()) {
+                    ldStr.append("Currently there are no active feeds in the Asterix");
+                } else {
+                    ldStr.append("Active Feeds");
+                }
+                FeedConnectionId feedId = null;
+                for (FeedActivity feedActivity : lfa) {
+                    feedId = new FeedConnectionId(feedActivity.getDataverseName(), feedActivity.getFeedName(),
+                            feedActivity.getDatasetName());
+                    ldStr.append("<br />");
+                    ldStr.append("<br />");
+                    ldStr.append("<a href=\"/feed/dashboard?dataverse=" + feedActivity.getDataverseName() + "&feed="
+                            + feedActivity.getFeedName() + "&dataset=" + feedActivity.getDatasetName() + "\">" + feedId
+                            + "</a>");
+                    ldStr.append("<br />");
+                }
+
+                outStr = String.format(sb.toString(), ldStr.toString());
+                MetadataManager.INSTANCE.commitTransaction(ctx);
+
+            }
+
+            PrintWriter out = response.getWriter();
+            out.println(outStr);
+        } catch (ACIDException | MetadataException e) {
+
+        }
+    }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java
new file mode 100644
index 0000000..433f834
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.asterix.api.http.servlet;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.RemoteSocketMessageListener;
+
+public class FeedServletUtil {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedServletUtil.class.getName());
+    private static final char EOL = (char) "\n".getBytes()[0];
+
+    public static void initiateSubscription(FeedConnectionId feedId, String host, int port) throws IOException {
+        LinkedBlockingQueue<String> outbox = new LinkedBlockingQueue<String>();
+        int subscriptionPort = port + 1;
+        Socket sc = new Socket(host, subscriptionPort);
+        InputStream in = sc.getInputStream();
+
+        CharBuffer buffer = CharBuffer.allocate(50);
+        char ch = 0;
+        while (ch != EOL) {
+            buffer.put(ch);
+            ch = (char) in.read();
+        }
+        buffer.flip();
+        String s = new String(buffer.array());
+        int feedSubscriptionPort = Integer.parseInt(s.trim());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Response from Super Feed Manager Report Service " + port + " will connect at " + host + " "
+                    + port);
+        }
+
+        // register the feed subscription queue with FeedLifecycleListener
+        FeedLifecycleListener.INSTANCE.registerFeedReportQueue(feedId, outbox);
+        RemoteSocketMessageListener listener = new RemoteSocketMessageListener(host, feedSubscriptionPort, outbox);
+        listener.start();
+    }
+}
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 772fad6..ac91636 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
@@ -30,9 +30,8 @@
     }
 
     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.COMPACT };
+        Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPDATE, Kind.DML_CMD_LIST,
+                Kind.LOAD_FROM_FILE, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET, Kind.COMPACT };
         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 929465a..c81808c 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
@@ -20,8 +20,13 @@
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 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;
@@ -32,19 +37,20 @@
 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.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -76,30 +82,38 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetNodegroupCardinalityHint;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 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.FeedDatasetDetails;
+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.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;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.types.TypeSignature;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.result.ResultReader;
 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.CompiledIndexCompactStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledInsertStatement;
@@ -128,6 +142,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
@@ -261,13 +277,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;
                 }
 
@@ -432,7 +457,11 @@
                             .getPartitioningExprs();
                     ARecordType aRecordType = (ARecordType) itemType;
                     aRecordType.validatePartitioningExpressions(partitioningExprs);
-                    String ngName = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
+
+                    Identifier ngNameId = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName();
+                    String ngName = ngNameId != null ? ngNameId.getValue() : configureNodegroupForDataset(dd,
+                            dataverseName, mdTxnCtx);
+     
                     String compactionPolicy = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getCompactionPolicy();
                     Map<String, String> compactionPolicyProperties = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
                             .getCompactionPolicyProperties();
@@ -453,37 +482,8 @@
                     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);
-                    String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
-                    String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterFactoryClassname();
-                    Map<String, String> configuration = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getConfiguration();
-                    FunctionSignature signature = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getFunctionSignature();
-                    String compactionPolicy = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getCompactionPolicy();
-                    Map<String, String> compactionPolicyProperties = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getCompactionPolicyProperties();
-                    if (compactionPolicy == null) {
-                        compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
-                        compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
-                    } else {
-                        validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx);
-                    }
-                    datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
-                            InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
-                            ngName, adapter, configuration, signature,
-                            FeedDatasetDetails.FeedState.INACTIVE.toString(), compactionPolicy,
-                            compactionPolicyProperties);
-                    break;
-                }
-            }
+
+              }
 
             //#. initialize DatasetIdFactory if it is not initialized.
             if (!DatasetIdFactory.isInitialized()) {
@@ -495,7 +495,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,
@@ -570,6 +570,58 @@
         }
     }
 
+    private String configureNodegroupForDataset(DatasetDecl dd, String dataverse, MetadataTransactionContext mdTxnCtx)
+            throws AsterixException {
+        int nodegroupCardinality = -1;
+        String nodegroupName;
+        String hintValue = dd.getHints().get(DatasetNodegroupCardinalityHint.NAME);
+        if (hintValue == null) {
+            nodegroupName = MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME;
+            return nodegroupName;
+        } else {
+            int numChosen = 0;
+            boolean valid = DatasetHints.validate(DatasetNodegroupCardinalityHint.NAME,
+                    dd.getHints().get(DatasetNodegroupCardinalityHint.NAME)).first;
+            if (!valid) {
+                throw new AsterixException("Incorrect use of hint:" + DatasetNodegroupCardinalityHint.NAME);
+            } else {
+                nodegroupCardinality = Integer.parseInt(dd.getHints().get(DatasetNodegroupCardinalityHint.NAME));
+            }
+            Set<String> nodeNames = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames();
+            Set<String> nodeNamesClone = new HashSet<String>();
+            for (String node : nodeNames) {
+                nodeNamesClone.add(node);
+            }
+            String metadataNodeName = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
+            List<String> selectedNodes = new ArrayList<String>();
+            selectedNodes.add(metadataNodeName);
+            numChosen++;
+            nodeNamesClone.remove(metadataNodeName);
+
+            if (numChosen < nodegroupCardinality) {
+                Random random = new Random();
+                String[] nodes = nodeNamesClone.toArray(new String[] {});
+                int[] b = new int[nodeNamesClone.size()];
+                for (int i = 0; i < b.length; i++) {
+                    b[i] = i;
+                }
+
+                for (int i = 0; i < nodegroupCardinality - numChosen; i++) {
+                    int selected = i + random.nextInt(nodeNamesClone.size() - i);
+                    int selNodeIndex = b[selected];
+                    selectedNodes.add(nodes[selNodeIndex]);
+                    int temp = b[0];
+                    b[0] = b[selected];
+                    b[selected] = temp;
+                }
+            }
+            nodegroupName = dataverse + ":" + dd.getName().getValue();
+            MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(nodegroupName, selectedNodes));
+            return nodegroupName;
+        }
+
+    }
+
     private void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
 
@@ -616,6 +668,19 @@
                 }
             }
 
+            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
             Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
                     stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false,
@@ -772,12 +837,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);
@@ -890,7 +965,22 @@
                 }
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            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) {
 
                 //#. prepare jobs to drop the datatset and the indexes in NC
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -915,6 +1005,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);
@@ -927,6 +1022,13 @@
 
             //#. finally, delete the dataset.
             MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+            // Drop the associated nodegroup
+            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);
+                }
+            }
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
@@ -991,7 +1093,20 @@
                         + dataverseName);
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            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) {
                 indexName = stmtIndexDrop.getIndexName().getValue();
                 Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 if (index == null) {
@@ -1307,57 +1422,183 @@
 
     }
 
-    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();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         acquireReadLatch();
-
+        boolean readLatchAcquired = true;
         try {
+            ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
+            String dataverseName = getActiveDataverseName(cfs.getDataverseName());
             metadataProvider.setWriteTransaction(true);
-            BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-            String dataverseName = getActiveDataverseName(bfs.getDataverseName());
 
-            CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName, bfs.getDatasetName()
-                    .getValue(), bfs.getQuery(), bfs.getVarCounter());
+            CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(), cfs
+                    .getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
 
-            Dataset dataset;
-            dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName, bfs
-                    .getDatasetName().getValue());
+            Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+                    dataverseName, cfs.getDatasetName().getValue());
             if (dataset == null) {
-                throw new AsterixException("Unknown dataset :" + bfs.getDatasetName().getValue());
+                throw new AsterixException("Unknown target dataset :" + cfs.getDatasetName().getValue());
+            }
+
+            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());
+            }
+
+            FeedConnectionId feedConnId = new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName()
+                    .getValue());
+            FeedActivity recentActivity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+                    feedConnId, null);
+            boolean isFeedActive = FeedUtil.isFeedActive(recentActivity);
+            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 AsterixException("Dataset " + cfs.getDatasetName().getValue() + " is not an interal dataset");
             }
-            bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
-            cbfs.setQuery(bfs.getQuery());
-            metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
 
-            JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
+            FeedPolicy feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName,
+                    cbfs.getPolicyName());
+            if (feedPolicy == null) {
+                feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx,
+                        MetadataConstants.METADATA_DATAVERSE_NAME, cbfs.getPolicyName());
+                if (feedPolicy == null) {
+                    throw new AsterixException("Unknown feed policy" + cbfs.getPolicyName());
+                }
+            }
+
+            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, cfs.getQuery(), cbfs);
+            JobSpecification newJobSpec = FeedUtil.alterJobSpecificationForFeed(compiled, feedConnId, feedPolicy);
+
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Altered feed ingestion spec to wrap operators");
+            }
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-
-            if (compiled != null) {
-                runJob(hcc, compiled, true);
+            String waitForCompletionParam = metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION);
+            boolean waitForCompletion = waitForCompletionParam == null ? false : Boolean
+                    .valueOf(waitForCompletionParam);
+            if (waitForCompletion) {
+                releaseReadLatch();
+                readLatchAcquired = false;
             }
-
+            runJob(hcc, newJobSpec, waitForCompletion);
         } catch (Exception e) {
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
             throw e;
         } finally {
-            releaseReadLatch();
+            if (readLatchAcquired) {
+                releaseReadLatch();
+            }
         }
     }
 
-    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;
@@ -1365,17 +1606,45 @@
         acquireReadLatch();
 
         try {
-            ControlFeedStatement cfs = (ControlFeedStatement) stmt;
+            DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
             String dataverseName = getActiveDataverseName(cfs.getDataverseName());
-            CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
-                    dataverseName, cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
-            JobSpecification jobSpec = FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider);
+
+            String datasetName = cfs.getDatasetName().getValue();
+            Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+                    dataverseName, cfs.getDatasetName().getValue());
+            if (dataset == null) {
+                throw new AsterixException("Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse "
+                        + dataverseName);
+            }
+            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().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) {
+                throw new AsterixException("Feed " + cfs.getDatasetName().getValue()
+                        + " is currently INACTIVE. Operation not supported");
+            }
+
+            CompiledDisconnectFeedStatement clcfs = new CompiledDisconnectFeedStatement(dataverseName, cfs
+                    .getFeedName().getValue(), cfs.getDatasetName().getValue());
+
+            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);
@@ -1405,7 +1674,7 @@
             if (ds == null) {
                 throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
                         + dataverseName + ".");
-            } else if (ds.getDatasetType() != DatasetType.INTERNAL && ds.getDatasetType() != DatasetType.FEED) {
+            } else if (ds.getDatasetType() != DatasetType.INTERNAL) {
                 throw new AlgebricksException("Cannot compact the extrenal dataset " + datasetName + ".");
             }
 
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 da0bebc..9a48c21 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
@@ -207,7 +207,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 c7ad8bd..5e52f3e 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
@@ -14,21 +14,11 @@
  */
 package edu.uci.ics.asterix.file;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
 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.external.feed.lifecycle.AlterFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage.MessageType;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
 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;
@@ -57,60 +47,17 @@
      * @throws AsterixException
      * @throws AlgebricksException
      */
-    public static JobSpecification buildControlFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
-            AqlMetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
-        switch (controlFeedStatement.getOperationType()) {
-            case ALTER:
-            case END: {
-                return createSendMessageToFeedJobSpec(controlFeedStatement, metadataProvider);
-            }
-            default: {
-                throw new AsterixException("Unknown Operation Type: " + controlFeedStatement.getOperationType());
-            }
-
-        }
-    }
-
-    private static JobSpecification createSendMessageToFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
-            AqlMetadataProvider metadataProvider) throws AsterixException {
-        String dataverseName = controlFeedStatement.getDataverseName() == null ? metadataProvider
-                .getDefaultDataverseName() : controlFeedStatement.getDataverseName();
-        String datasetName = controlFeedStatement.getDatasetName();
-        String datasetPath = dataverseName + File.separator + datasetName;
-
-        LOGGER.info(" DATASETPATH: " + datasetPath);
-
-        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;
 
-        List<IFeedMessage> feedMessages = new ArrayList<IFeedMessage>();
-        switch (controlFeedStatement.getOperationType()) {
-            case END:
-                feedMessages.add(new FeedMessage(MessageType.STOP));
-                break;
-            case ALTER:
-                feedMessages.add(new AlterFeedMessage(controlFeedStatement.getProperties()));
-                break;
-        }
-
         try {
-            Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildFeedMessengerRuntime(
-                    metadataProvider, spec, (FeedDatasetDetails) dataset.getDatasetDetails(), dataverseName,
-                    datasetName, feedMessages);
+            Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider
+                    .buildDisconnectFeedMessengerRuntime(spec, dataverseName, feedName, datasetName, feedActivity);
             feedMessenger = p.first;
             messengerPc = p.second;
         } catch (AlgebricksException e) {
@@ -121,9 +68,7 @@
 
         NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
-
         spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
-
         spec.addRoot(nullSink);
         return spec;
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index dc201dd..c2f9192 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -23,6 +23,9 @@
 
 import edu.uci.ics.asterix.api.http.servlet.APIServlet;
 import edu.uci.ics.asterix.api.http.servlet.DDLAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedDashboardServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedDataProviderServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedServlet;
 import edu.uci.ics.asterix.api.http.servlet.QueryAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.QueryResultAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.QueryStatusAPIServlet;
@@ -46,6 +49,8 @@
 
     private Server webServer;
     private Server jsonAPIServer;
+    private Server feedServer;
+
     private static IAsterixStateProxy proxy;
     private ICCApplicationContext appCtx;
 
@@ -56,19 +61,30 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting Asterix cluster controller");
         }
+
         appCtx.setThreadFactory(AsterixThreadFactory.INSTANCE);
-        AsterixAppContextInfo.initialize(appCtx);
+        AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection());
+
         proxy = AsterixStateProxy.registerRemoteObject();
         appCtx.setDistributedState(proxy);
 
         AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
         MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
 
+        AsterixAppContextInfo.getInstance().getCCApplicationContext()
+                .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+
         AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
         setupWebServer(externalProperties);
         webServer.start();
+
         setupJSONAPIServer(externalProperties);
         jsonAPIServer.start();
+        ExternalLibraryBootstrap.setUpExternaLibraries(false);
+
+        setupFeedServer(externalProperties);
+        feedServer.start();
+
         ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
     }
 
@@ -119,4 +135,21 @@
         context.addServlet(new ServletHolder(new UpdateAPIServlet()), "/update");
         context.addServlet(new ServletHolder(new DDLAPIServlet()), "/ddl");
     }
+
+    private void setupFeedServer(AsterixExternalProperties externalProperties) throws Exception {
+        feedServer = new Server(externalProperties.getFeedServerPort());
+
+        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+        context.setContextPath("/");
+
+        IHyracksClientConnection hcc = getNewHyracksClientConnection();
+        context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+
+        feedServer.setHandler(context);
+        context.addServlet(new ServletHolder(new FeedServlet()), "/");
+        context.addServlet(new ServletHolder(new FeedDashboardServlet()), "/feed/dashboard");
+        context.addServlet(new ServletHolder(new FeedDataProviderServlet()), "/feed/data");
+
+        // add paths here
+    }
 }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index a864e61..6461b0c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -14,22 +14,53 @@
  */
 package edu.uci.ics.asterix.hyracks.bootstrap;
 
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse.Status;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWorkResponse;
 import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.hyracks.api.application.IClusterLifecycleListener;
 
 public class ClusterLifecycleListener implements IClusterLifecycleListener {
 
+    private static final Logger LOGGER = Logger.getLogger(ClusterLifecycleListener.class.getName());
+
+    private static final LinkedBlockingQueue<Set<IClusterManagementWork>> workRequestQueue = new LinkedBlockingQueue<Set<IClusterManagementWork>>();
+
+    private static ClusterWorkExecutor eventHandler = new ClusterWorkExecutor(workRequestQueue);
+
+    private static List<IClusterManagementWorkResponse> pendingWorkResponses = new ArrayList<IClusterManagementWorkResponse>();
+
     public static ClusterLifecycleListener INSTANCE = new ClusterLifecycleListener();
 
     private ClusterLifecycleListener() {
+        Thread t = new Thread(eventHandler);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Starting cluster event handler");
+        }
+        t.start();
     }
 
-    private static final Logger LOGGER = Logger.getLogger(ClusterLifecycleListener.class.getName());
+    public enum ClusterEventType {
+        NODE_JOIN,
+        NODE_FAILURE
+    }
 
     @Override
     public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) {
@@ -37,6 +68,21 @@
             LOGGER.info("NC: " + nodeId + " joined");
         }
         AsterixClusterProperties.INSTANCE.addNCConfiguration(nodeId, ncConfiguration);
+        Set<String> nodeAddition = new HashSet<String>();
+        nodeAddition.add(nodeId);
+        updateProgress(ClusterEventType.NODE_JOIN, nodeAddition);
+        Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();
+        Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+        for (IClusterEventsSubscriber sub : subscribers) {
+            Set<IClusterManagementWork> workRequest = sub.notifyNodeJoin(nodeId);
+            if (workRequest != null && !workRequest.isEmpty()) {
+                work.addAll(workRequest);
+            }
+        }
+        if (!work.isEmpty()) {
+            executeWorkSet(work);
+        }
+
     }
 
     public void notifyNodeFailure(Set<String> deadNodeIds) {
@@ -46,7 +92,123 @@
             }
             AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
         }
-
+        updateProgress(ClusterEventType.NODE_FAILURE, deadNodeIds);
+        Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();
+        Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+        for (IClusterEventsSubscriber sub : subscribers) {
+            Set<IClusterManagementWork> workRequest = sub.notifyNodeFailure(deadNodeIds);
+            if (workRequest != null && !workRequest.isEmpty()) {
+                work.addAll(workRequest);
+            }
+        }
+        if (!work.isEmpty()) {
+            executeWorkSet(work);
+        }
     }
 
+    private void submitWork(Set<IClusterManagementWork> work) {
+        try {
+            workRequestQueue.put(work);
+        } catch (InterruptedException e) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Interrupted :" + e.getMessage());
+            }
+        }
+    }
+
+    private void updateProgress(ClusterEventType eventType, Set<String> nodeIds) {
+        List<IClusterManagementWorkResponse> completedResponses = new ArrayList<IClusterManagementWorkResponse>();
+        boolean isComplete = false;
+        for (IClusterManagementWorkResponse resp : pendingWorkResponses) {
+            switch (eventType) {
+                case NODE_FAILURE:
+                    isComplete = ((RemoveNodeWorkResponse) resp).updateProgress(nodeIds);
+                    if (isComplete) {
+                        resp.setStatus(Status.SUCCESS);
+                        resp.getWork().getSourceSubscriber().notifyRequestCompletion(resp);
+                        completedResponses.add(resp);
+                    }
+                    break;
+
+                case NODE_JOIN:
+                    isComplete = ((AddNodeWorkResponse) resp).updateProgress(nodeIds.iterator().next());
+                    if (isComplete) {
+                        resp.setStatus(Status.SUCCESS);
+                        resp.getWork().getSourceSubscriber().notifyRequestCompletion(resp);
+                        completedResponses.add(resp);
+                    }
+                    break;
+            }
+        }
+        pendingWorkResponses.removeAll(completedResponses);
+    }
+
+    private void executeWorkSet(Set<IClusterManagementWork> workSet) {
+        int nodesToAdd = 0;
+        Set<String> nodesToRemove = new HashSet<String>();
+        Set<AddNodeWork> nodeAdditionRequests = new HashSet<AddNodeWork>();
+        Set<IClusterManagementWork> nodeRemovalRequests = new HashSet<IClusterManagementWork>();
+        for (IClusterManagementWork w : workSet) {
+            switch (w.getClusterManagementWorkType()) {
+                case ADD_NODE:
+                    if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
+                        nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+                    }
+                    nodeAdditionRequests.add((AddNodeWork) w);
+                    break;
+                case REMOVE_NODE:
+                    nodesToRemove.addAll(((RemoveNodeWork) w).getNodesToBeRemoved());
+                    nodeRemovalRequests.add(w);
+                    RemoveNodeWorkResponse response = new RemoveNodeWorkResponse((RemoveNodeWork) w, Status.IN_PROGRESS);
+                    pendingWorkResponses.add(response);
+                    break;
+            }
+        }
+
+        List<String> addedNodes = new ArrayList<String>();
+        String asterixInstanceName = AsterixClusterProperties.INSTANCE.getCluster().getInstanceName();
+        for (int i = 0; i < nodesToAdd; i++) {
+            Node node = AsterixClusterProperties.INSTANCE.getAvailableSubstitutionNode();
+            if (node != null) {
+                try {
+                    ClusterManager.INSTANCE.addNode(node);
+                    addedNodes.add(asterixInstanceName + "_" + node.getId());
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Added NC at:" + node.getId());
+                    }
+                } catch (AsterixException e) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Unable to add NC at:" + node.getId());
+                    }
+                    e.printStackTrace();
+                }
+            } else {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Unable to add NC: no more available nodes");
+                }
+
+            }
+        }
+
+        for (AddNodeWork w : nodeAdditionRequests) {
+            int n = w.getNumberOfNodes();
+            List<String> nodesToBeAddedForWork = new ArrayList<String>();
+            for (int i = 0; i < n && i < addedNodes.size(); i++) {
+                nodesToBeAddedForWork.add(addedNodes.get(i));
+            }
+            if (nodesToBeAddedForWork.isEmpty()) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Unable to satisfy request by " + w);
+                }
+                AddNodeWorkResponse response = new AddNodeWorkResponse(w, nodesToBeAddedForWork);
+                response.setStatus(Status.FAILURE);
+                w.getSourceSubscriber().notifyRequestCompletion(response);
+
+            } else {
+                AddNodeWorkResponse response = new AddNodeWorkResponse(w, nodesToBeAddedForWork);
+                pendingWorkResponses.add(response);
+            }
+        }
+
+    }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
new file mode 100644
index 0000000..a6c1f8b
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
@@ -0,0 +1,106 @@
+/*
+ * 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.hyracks.bootstrap;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse.Status;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+public class ClusterWorkExecutor implements Runnable {
+
+    private static final Logger LOGGER = Logger.getLogger(ClusterWorkExecutor.class.getName());
+
+    private final LinkedBlockingQueue<Set<IClusterManagementWork>> inbox;
+
+    public ClusterWorkExecutor(LinkedBlockingQueue<Set<IClusterManagementWork>> inbox) {
+        this.inbox = inbox;
+    }
+    
+   
+    @Override
+    public void run() {
+        while (true) {
+            try {
+                Set<IClusterManagementWork> workSet = inbox.take();
+                int nodesToAdd = 0;
+                Set<String> nodesToRemove = new HashSet<String>();
+                Set<IClusterManagementWork> nodeAdditionRequests = new HashSet<IClusterManagementWork>();
+                Set<IClusterManagementWork> nodeRemovalRequests = new HashSet<IClusterManagementWork>();
+                for (IClusterManagementWork w : workSet) {
+                    switch (w.getClusterManagementWorkType()) {
+                        case ADD_NODE:
+                            if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
+                                nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+                            }
+                            nodeAdditionRequests.add(w);
+                            break;
+                        case REMOVE_NODE:
+                            nodesToRemove.addAll(((RemoveNodeWork) w).getNodesToBeRemoved());
+                            nodeRemovalRequests.add(w);
+                            break;
+                    }
+                }
+
+                Set<Node> addedNodes = new HashSet<Node>();
+                for (int i = 0; i < nodesToAdd; i++) {
+                    Node node = AsterixClusterProperties.INSTANCE.getAvailableSubstitutionNode();
+                    if (node != null) {
+                        try {
+                            ClusterManager.INSTANCE.addNode(node);
+                            addedNodes.add(node);
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info("Added NC at:" + node.getId());
+                            }
+                        } catch (AsterixException e) {
+                            if (LOGGER.isLoggable(Level.WARNING)) {
+                                LOGGER.warning("Unable to add NC at:" + node.getId());
+                            }
+                            e.printStackTrace();
+                        }
+                    } else {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Unable to add NC: no more available nodes");
+                        }
+                    }
+                }
+                
+
+            } catch (InterruptedException e) {
+                if (LOGGER.isLoggable(Level.SEVERE)) {
+                    LOGGER.severe("interruped" + e.getMessage());
+                }
+                throw new IllegalStateException(e);
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.SEVERE)) {
+                    LOGGER.severe("Unexpected exception in handling cluster event" + e.getMessage());
+                }
+            }
+
+        }
+    }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
new file mode 100755
index 0000000..ecf1a14e
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
@@ -0,0 +1,300 @@
+/*
+ * Copyright 2009-2012 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.hyracks.bootstrap;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.external.library.ExternalLibraryManager;
+import edu.uci.ics.asterix.external.library.Function;
+import edu.uci.ics.asterix.external.library.Functions;
+import edu.uci.ics.asterix.external.library.Library;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
+
+public class ExternalLibraryBootstrap {
+
+    public static void setUpExternaLibraries(boolean isMetadataNode) throws Exception {
+
+        Map<String, List<String>> uninstalledLibs = null;
+        if (isMetadataNode) {
+            uninstalledLibs = uninstallLibraries();
+        }
+
+        File installLibDir = getLibraryInstallDir();
+        if (installLibDir.exists()) {
+            for (String dataverse : installLibDir.list()) {
+                File dataverseDir = new File(installLibDir, dataverse);
+                String[] libraries = dataverseDir.list();
+                for (String library : libraries) {
+                    registerLibrary(dataverse, library, isMetadataNode, installLibDir);
+                    if (isMetadataNode) {
+                        File libraryDir = new File(installLibDir.getAbsolutePath() + File.separator + dataverse
+                                + File.separator + library);
+                        installLibraryIfNeeded(dataverse, libraryDir, uninstalledLibs);
+                    }
+                }
+            }
+        }
+    }
+
+    private static Map<String, List<String>> uninstallLibraries() throws Exception {
+        Map<String, List<String>> uninstalledLibs = new HashMap<String, List<String>>();
+        File uninstallLibDir = getLibraryUninstallDir();
+        String[] uninstallLibNames;
+        if (uninstallLibDir.exists()) {
+            uninstallLibNames = uninstallLibDir.list();
+            for (String uninstallLibName : uninstallLibNames) {
+                String[] components = uninstallLibName.split("\\.");
+                String dataverse = components[0];
+                String libName = components[1];
+                uninstallLibrary(dataverse, libName);
+                new File(uninstallLibDir, uninstallLibName).delete();
+                List<String> uinstalledLibsInDv = uninstalledLibs.get(dataverse);
+                if (uinstalledLibsInDv == null) {
+                    uinstalledLibsInDv = new ArrayList<String>();
+                    uninstalledLibs.put(dataverse, uinstalledLibsInDv);
+                }
+                uinstalledLibsInDv.add(libName);
+            }
+        }
+        return uninstalledLibs;
+    }
+
+    private static boolean uninstallLibrary(String dataverse, String libraryName) throws AsterixException,
+            RemoteException, ACIDException {
+        MetadataTransactionContext mdTxnCtx = null;
+        try {
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+            if (dv == null) {
+                return false;
+            }
+
+            edu.uci.ics.asterix.metadata.entities.Library library = MetadataManager.INSTANCE.getLibrary(mdTxnCtx,
+                    dataverse, libraryName);
+            if (library == null) {
+                return false;
+            }
+
+            List<edu.uci.ics.asterix.metadata.entities.Function> functions = MetadataManager.INSTANCE
+                    .getDataverseFunctions(mdTxnCtx, dataverse);
+            for (edu.uci.ics.asterix.metadata.entities.Function function : functions) {
+                if (function.getName().startsWith(libraryName + ":")) {
+                    MetadataManager.INSTANCE.dropFunction(mdTxnCtx, new FunctionSignature(dataverse,
+                            function.getName(), function.getArity()));
+                }
+            }
+
+            MetadataManager.INSTANCE.dropLibrary(mdTxnCtx, dataverse, libraryName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AsterixException(e);
+        }
+        return true;
+    }
+
+    // Each element of a library is installed as part of a transaction. Any
+    // failure in installing an element does not effect installation of other
+    // libraries
+    private static void installLibraryIfNeeded(String dataverse, final File libraryDir,
+            Map<String, List<String>> uninstalledLibs) throws Exception {
+
+        String libraryName = libraryDir.getName();
+        List<String> uninstalledLibsInDv = uninstalledLibs.get(dataverse);
+        boolean wasUninstalled = uninstalledLibsInDv != null && uninstalledLibsInDv.contains(libraryName);
+
+        MetadataTransactionContext mdTxnCtx = null;
+        try {
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            edu.uci.ics.asterix.metadata.entities.Library libraryInMetadata = MetadataManager.INSTANCE.getLibrary(
+                    mdTxnCtx, dataverse, libraryName);
+            if (libraryInMetadata != null && !wasUninstalled) {
+                return;
+            }
+
+            String[] libraryDescriptors = libraryDir.list(new FilenameFilter() {
+                @Override
+                public boolean accept(File dir, String name) {
+                    return name.endsWith(".xml");
+                }
+            });
+
+            Library library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
+
+            if (libraryDescriptors.length == 0) {
+                throw new Exception("No library descriptors defined");
+            } else if (libraryDescriptors.length > 1) {
+                throw new Exception("More than 1 library descriptors defined");
+            }
+
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+            if (dv == null) {
+                MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverse,
+                        NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, IMetadataEntity.PENDING_NO_OP));
+            }
+            for (Function function : library.getFunctions().getFunction()) {
+                String[] fargs = function.getArguments().trim().split(",");
+                List<String> args = new ArrayList<String>();
+                for (String arg : fargs) {
+                    args.add(arg);
+                }
+                edu.uci.ics.asterix.metadata.entities.Function f = new edu.uci.ics.asterix.metadata.entities.Function(
+                        dataverse, libraryName + "#" + function.getName(), args.size(), args, function.getReturnType(),
+                        function.getDefinition(), library.getLanguage(), function.getFunctionType());
+                MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
+            }
+
+            MetadataManager.INSTANCE.addLibrary(mdTxnCtx, new edu.uci.ics.asterix.metadata.entities.Library(dataverse,
+                    libraryName));
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            e.printStackTrace();
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+        }
+    }
+
+    private static void registerLibrary(String dataverse, String libraryName, boolean isMetadataNode, File installLibDir)
+            throws Exception {
+        ClassLoader classLoader = getLibraryClassLoader(dataverse, libraryName);
+        ExternalLibraryManager.registerLibraryClassLoader(dataverse, libraryName, classLoader);
+    }
+
+    private static Library getLibrary(File libraryXMLPath) throws Exception {
+        JAXBContext configCtx = JAXBContext.newInstance(Library.class);
+        Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+        Library library = (Library) unmarshaller.unmarshal(libraryXMLPath);
+        return library;
+    }
+
+    private static ClassLoader getLibraryClassLoader(String dataverse, String libraryName) throws Exception {
+        System.out.println(" installing lirbary " + libraryName + " in dataverse " + dataverse);
+        File installDir = getLibraryInstallDir();
+        System.out.println(" install directory " + installDir.getAbsolutePath());
+
+        File libDir = new File(installDir.getAbsolutePath() + File.separator + dataverse + File.separator + libraryName);
+        FilenameFilter jarFileFilter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return name.endsWith(".jar");
+            }
+        };
+
+        String[] jarsInLibDir = libDir.list(jarFileFilter);
+        System.out.println(" jars in lib dir " + jarsInLibDir);
+
+        if (jarsInLibDir.length > 1) {
+            throw new Exception("Incorrect library structure: found multiple library jars");
+        }
+        if (jarsInLibDir.length < 0) {
+            throw new Exception("Incorrect library structure: could not find library jar");
+        }
+
+        File libJar = new File(libDir, jarsInLibDir[0]);
+        File libDependencyDir = new File(libDir.getAbsolutePath() + File.separator + "lib");
+        int numDependencies = 1;
+        String[] libraryDependencies = null;
+        if (libDependencyDir.exists()) {
+            libraryDependencies = libDependencyDir.list(jarFileFilter);
+            numDependencies += libraryDependencies.length;
+        }
+
+        ClassLoader parentClassLoader = ExternalLibraryBootstrap.class.getClassLoader();
+        URL[] urls = new URL[numDependencies];
+        int count = 0;
+        urls[count++] = libJar.toURL();
+
+        if (libraryDependencies != null && libraryDependencies.length > 0) {
+            for (String dependency : libraryDependencies) {
+                File file = new File(libDependencyDir + File.separator + dependency);
+                urls[count++] = file.toURL();
+            }
+        }
+        ClassLoader classLoader = new URLClassLoader(urls, parentClassLoader);
+        return classLoader;
+    }
+
+    private static File getLibraryInstallDir() {
+        String workingDir = System.getProperty("user.dir");
+        return new File(workingDir + File.separator + "library");
+    }
+
+    private static File getLibraryUninstallDir() {
+        String workingDir = System.getProperty("user.dir");
+        return new File(workingDir + File.separator + "uninstall");
+    }
+
+}
+
+class ExternalLibrary {
+
+    private final String dataverse;
+    private final String name;
+    private final String language;
+    private final Functions functions;
+
+    public ExternalLibrary(String dataverse, String name, String language, Functions functions) {
+        this.dataverse = dataverse;
+        this.name = name;
+        this.language = language;
+        this.functions = functions;
+    }
+
+    public String toString() {
+        StringBuilder builder = new StringBuilder("");
+        builder.append("Library");
+        builder.append("\n");
+        builder.append("Functions");
+        builder.append("\n");
+        for (Function function : functions.getFunction()) {
+            builder.append(function);
+            builder.append("\n");
+        }
+        return new String(builder);
+    }
+
+    public String getDataverse() {
+        return dataverse;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getLanguage() {
+        return language;
+    }
+
+    public Functions getFunctions() {
+        return functions;
+    }
+
+}
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
new file mode 100644
index 0000000..37c6e8a
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
@@ -0,0 +1,1172 @@
+/*
+ * 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.hyracks.bootstrap;
+
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+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.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;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.file.JobSpecificationUtils;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailure.FailureType;
+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.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+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.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedManagerElectMessage;
+import edu.uci.ics.asterix.metadata.feeds.FeedMetaOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener.IMessageAnalyzer;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager;
+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.asterix.runtime.formats.NonTaggedDataFormat;
+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.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;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobInfo;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+
+public class FeedLifecycleListener implements IJobLifecycleListener, IClusterEventsSubscriber, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(FeedLifecycleListener.class.getName());
+
+    public static FeedLifecycleListener INSTANCE = new FeedLifecycleListener();
+
+    public static final int FEED_HEALTH_PORT = 2999;
+
+    private LinkedBlockingQueue<Message> jobEventInbox;
+    private LinkedBlockingQueue<IClusterManagementWorkResponse> responseInbox;
+    private Map<FeedInfo, List<String>> dependentFeeds = new HashMap<FeedInfo, List<String>>();
+    private IMessageAnalyzer healthDataParser;
+    private MessageListener feedHealthDataListener;
+    private ExecutorService executorService = Executors.newCachedThreadPool();
+    private Map<FeedConnectionId, LinkedBlockingQueue<String>> feedReportQueue = new HashMap<FeedConnectionId, LinkedBlockingQueue<String>>();
+    private State state;
+
+    private FeedLifecycleListener() {
+        jobEventInbox = new LinkedBlockingQueue<Message>();
+        feedJobNotificationHandler = new FeedJobNotificationHandler(jobEventInbox);
+        responseInbox = new LinkedBlockingQueue<IClusterManagementWorkResponse>();
+        feedWorkRequestResponseHandler = new FeedWorkRequestResponseHandler(responseInbox);
+        this.healthDataParser = new FeedHealthDataParser();
+        feedHealthDataListener = new MessageListener(FEED_HEALTH_PORT, healthDataParser.getMessageQueue());
+        try {
+            feedHealthDataListener.start();
+        } catch (Exception e) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Unable to start Feed health data listener");
+            }
+        }
+        executorService.execute(feedJobNotificationHandler);
+        executorService.execute(feedWorkRequestResponseHandler);
+        ClusterManager.INSTANCE.registerSubscriber(this);
+        state = AsterixClusterProperties.INSTANCE.getState();
+
+    }
+
+    private final FeedJobNotificationHandler feedJobNotificationHandler;
+    private final FeedWorkRequestResponseHandler feedWorkRequestResponseHandler;
+
+    @Override
+    public void notifyJobStart(JobId jobId) throws HyracksException {
+        if (feedJobNotificationHandler.isRegisteredFeed(jobId)) {
+            jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_START));
+        }
+    }
+
+    @Override
+    public void notifyJobFinish(JobId jobId) throws HyracksException {
+        if (feedJobNotificationHandler.isRegisteredFeed(jobId)) {
+            jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_FINISH));
+        }
+    }
+
+    @Override
+    public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
+
+        JobSpecification spec = acggf.getJobSpecification();
+        boolean feedIngestionJob = false;
+        FeedConnectionId feedId = null;
+        Map<String, String> feedPolicy = null;
+        for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
+            if (!(opDesc instanceof FeedIntakeOperatorDescriptor)) {
+                continue;
+            }
+            feedId = ((FeedIntakeOperatorDescriptor) opDesc).getFeedId();
+            feedPolicy = ((FeedIntakeOperatorDescriptor) opDesc).getFeedPolicy();
+            feedIngestionJob = true;
+            break;
+        }
+        if (feedIngestionJob) {
+            feedJobNotificationHandler.registerFeed(feedId, jobId, spec, feedPolicy);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Registered feed: " + feedId + " ingestion policy "
+                        + feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+            }
+        }
+
+    }
+
+    public void registerFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
+        feedReportQueue.put(feedId, queue);
+    }
+
+    public void deregisterFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
+        feedReportQueue.remove(feedId);
+    }
+
+    public LinkedBlockingQueue<String> getFeedReportQueue(FeedConnectionId feedId) {
+        return feedReportQueue.get(feedId);
+    }
+
+    private static class Message {
+        public JobId jobId;
+
+        public enum MessageKind {
+            JOB_START,
+            JOB_FINISH
+        }
+
+        public MessageKind messageKind;
+
+        public Message(JobId jobId, MessageKind msgKind) {
+            this.jobId = jobId;
+            this.messageKind = msgKind;
+        }
+    }
+
+    public static class FeedFailureReport {
+        public Map<FeedInfo, List<FeedFailure>> failures = new HashMap<FeedInfo, List<FeedFailure>>();
+
+        @Override
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            for (Map.Entry<FeedLifecycleListener.FeedInfo, List<FeedLifecycleListener.FeedFailure>> entry : failures
+                    .entrySet()) {
+                builder.append(entry.getKey() + " -> failures");
+                for (FeedFailure failure : entry.getValue()) {
+                    builder.append("failure -> " + failure);
+                }
+            }
+            return builder.toString();
+        }
+    }
+
+    private static class FeedHealthDataParser implements IMessageAnalyzer {
+
+        private LinkedBlockingQueue<String> inbox = new LinkedBlockingQueue<String>();
+
+        @Override
+        public LinkedBlockingQueue<String> getMessageQueue() {
+            return inbox;
+        }
+
+    }
+
+    private static class FeedJobNotificationHandler implements Runnable, Serializable {
+
+        private static final long serialVersionUID = 1L;
+        private LinkedBlockingQueue<Message> inbox;
+        private Map<JobId, FeedInfo> registeredFeeds = new HashMap<JobId, FeedInfo>();
+        private FeedMessenger feedMessenger;
+        private LinkedBlockingQueue<FeedMessengerMessage> messengerOutbox;
+        private int superFeedManagerPort = 3000;
+
+        public FeedJobNotificationHandler(LinkedBlockingQueue<Message> inbox) {
+            this.inbox = inbox;
+            messengerOutbox = new LinkedBlockingQueue<FeedMessengerMessage>();
+            feedMessenger = new FeedMessenger(messengerOutbox);
+            (new Thread(feedMessenger)).start();
+        }
+
+        public boolean isRegisteredFeed(JobId jobId) {
+            return registeredFeeds.containsKey(jobId);
+        }
+
+        public void registerFeed(FeedConnectionId feedId, JobId jobId, JobSpecification jobSpec,
+                Map<String, String> feedPolicy) {
+            if (registeredFeeds.containsKey(jobId)) {
+                throw new IllegalStateException(" Feed already registered ");
+            }
+            registeredFeeds.put(jobId, new FeedInfo(feedId, jobSpec, feedPolicy, jobId));
+        }
+
+        public void deregisterFeed(JobId jobId) {
+            FeedInfo feedInfo = registeredFeeds.remove(jobId);
+            if (feedInfo != null) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("DeRegistered Feed Info :" + feedInfo);
+                }
+            }
+        }
+
+        public void deregisterFeed(FeedInfo feedInfo) {
+            JobId jobId = feedInfo.jobId;
+            deregisterFeed(jobId);
+        }
+
+        @Override
+        public void run() {
+            Message mesg;
+            while (true) {
+                try {
+                    mesg = inbox.take();
+                    FeedInfo feedInfo = registeredFeeds.get(mesg.jobId);
+                    switch (mesg.messageKind) {
+                        case JOB_START:
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                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.feedConnectionId);
+                            }
+                            handleJobFinishMessage(feedInfo, mesg);
+                            deregisterFeed(mesg.jobId);
+                            break;
+                    }
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+
+            }
+        }
+
+        private void handleJobStartMessage(FeedInfo feedInfo, Message message) {
+
+            JobSpecification jobSpec = feedInfo.jobSpec;
+
+            List<OperatorDescriptorId> ingestOperatorIds = new ArrayList<OperatorDescriptorId>();
+            List<OperatorDescriptorId> computeOperatorIds = new ArrayList<OperatorDescriptorId>();
+            List<OperatorDescriptorId> storageOperatorIds = new ArrayList<OperatorDescriptorId>();
+
+            Map<OperatorDescriptorId, IOperatorDescriptor> operators = jobSpec.getOperatorMap();
+            for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
+                IOperatorDescriptor opDesc = entry.getValue();
+                IOperatorDescriptor actualOp = null;
+                if (opDesc instanceof FeedMetaOperatorDescriptor) {
+                    actualOp = ((FeedMetaOperatorDescriptor) opDesc).getCoreOperator();
+                } else {
+                    actualOp = opDesc;
+                }
+
+                if (actualOp instanceof AlgebricksMetaOperatorDescriptor) {
+                    AlgebricksMetaOperatorDescriptor op = ((AlgebricksMetaOperatorDescriptor) actualOp);
+                    IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
+                    for (IPushRuntimeFactory rf : runtimeFactories) {
+                        if (rf instanceof AssignRuntimeFactory) {
+                            computeOperatorIds.add(entry.getKey());
+                        }
+                    }
+                } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
+                    storageOperatorIds.add(entry.getKey());
+                } else if (actualOp instanceof FeedIntakeOperatorDescriptor) {
+                    ingestOperatorIds.add(entry.getKey());
+                }
+            }
+
+            try {
+                IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+                JobInfo info = hcc.getJobInfo(message.jobId);
+                feedInfo.jobInfo = info;
+                Map<String, String> feedActivityDetails = new HashMap<String, String>();
+                StringBuilder ingestLocs = new StringBuilder();
+                for (OperatorDescriptorId ingestOpId : ingestOperatorIds) {
+                    feedInfo.ingestLocations.addAll(info.getOperatorLocations().get(ingestOpId));
+                }
+                StringBuilder computeLocs = new StringBuilder();
+                for (OperatorDescriptorId computeOpId : computeOperatorIds) {
+                    List<String> locations = info.getOperatorLocations().get(computeOpId);
+                    if (locations != null) {
+                        feedInfo.computeLocations.addAll(locations);
+                    } else {
+                        feedInfo.computeLocations.addAll(feedInfo.ingestLocations);
+                    }
+                }
+                StringBuilder storageLocs = new StringBuilder();
+                for (OperatorDescriptorId storageOpId : storageOperatorIds) {
+                    feedInfo.storageLocations.addAll(info.getOperatorLocations().get(storageOpId));
+                }
+
+                for (String ingestLoc : feedInfo.ingestLocations) {
+                    ingestLocs.append(ingestLoc);
+                    ingestLocs.append(",");
+                }
+                if (ingestLocs.length() > 1) {
+                    ingestLocs.deleteCharAt(ingestLocs.length() - 1);
+                }
+                for (String computeLoc : feedInfo.computeLocations) {
+                    computeLocs.append(computeLoc);
+                    computeLocs.append(",");
+                }
+                if (computeLocs.length() > 1) {
+                    computeLocs.deleteCharAt(computeLocs.length() - 1);
+                }
+                for (String storageLoc : feedInfo.storageLocations) {
+                    storageLocs.append(storageLoc);
+                    storageLocs.append(",");
+                }
+                if (storageLocs.length() > 1) {
+                    storageLocs.deleteCharAt(storageLocs.length() - 1);
+                }
+
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.INGEST_LOCATIONS, ingestLocs.toString());
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.COMPUTE_LOCATIONS, computeLocs.toString());
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.STORAGE_LOCATIONS, storageLocs.toString());
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_POLICY_NAME,
+                        feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+
+                int superFeedManagerIndex = new Random().nextInt(feedInfo.ingestLocations.size());
+                String superFeedManagerHost = feedInfo.ingestLocations.get(superFeedManagerIndex);
+
+                Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+                String instanceName = cluster.getInstanceName();
+                String node = superFeedManagerHost.substring(instanceName.length() + 1);
+                String hostIp = null;
+                for (Node n : cluster.getNode()) {
+                    if (n.getId().equals(node)) {
+                        hostIp = n.getClusterIp();
+                        break;
+                    }
+                }
+                if (hostIp == null) {
+                    throw new IllegalStateException("Unknown node " + superFeedManagerHost);
+                }
+
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_HOST, hostIp);
+                feedActivityDetails.put(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_PORT, ""
+                        + superFeedManagerPort);
+
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Super Feed Manager for " + feedInfo.feedConnectionId + " is " + hostIp + " node "
+                            + superFeedManagerHost);
+                }
+
+                FeedManagerElectMessage feedMessage = new FeedManagerElectMessage(hostIp, superFeedManagerHost,
+                        superFeedManagerPort, feedInfo.feedConnectionId);
+                superFeedManagerPort += SuperFeedManager.PORT_RANGE_ASSIGNED;
+                messengerOutbox.add(new FeedMessengerMessage(feedMessage, feedInfo));
+                MetadataManager.INSTANCE.acquireWriteLatch();
+                MetadataTransactionContext mdTxnCtx = null;
+                try {
+                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                    FeedActivity fa = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+                            feedInfo.feedConnectionId, null);
+                    FeedActivityType nextState = FeedActivityType.FEED_BEGIN;
+                    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);
+                } finally {
+                    MetadataManager.INSTANCE.releaseWriteLatch();
+                }
+            } catch (Exception e) {
+                // TODO Add Exception handling here
+            }
+
+        }
+
+        private void handleJobFinishMessage(FeedInfo feedInfo, Message message) {
+            MetadataManager.INSTANCE.acquireWriteLatch();
+            MetadataTransactionContext mdTxnCtx = null;
+            boolean feedFailedDueToPostSubmissionNodeLoss = verfyReasonForFailure(feedInfo);
+            if (!feedFailedDueToPostSubmissionNodeLoss) {
+                try {
+                    IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+                    JobInfo info = hcc.getJobInfo(message.jobId);
+                    JobStatus status = info.getPendingStatus();
+                    List<Exception> exceptions;
+                    boolean failure = status != null && status.equals(JobStatus.FAILURE);
+                    FeedActivityType activityType = FeedActivityType.FEED_END;
+                    Map<String, String> details = new HashMap<String, String>();
+                    if (failure) {
+                        exceptions = info.getPendingExceptions();
+                        activityType = FeedActivityType.FEED_FAILURE;
+                        details.put(FeedActivity.FeedActivityDetails.EXCEPTION_MESSAGE, exceptions.get(0).getMessage());
+                    }
+                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                    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 {
+                        MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                    } catch (RemoteException | ACIDException ae) {
+                        throw new IllegalStateException(" Unable to abort ");
+                    }
+                } catch (Exception e) {
+                    // add exception handling here
+                } finally {
+                    MetadataManager.INSTANCE.releaseWriteLatch();
+                }
+            } else {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Attempt to revive feed");
+                }
+                FeedsActivator activator = new FeedsActivator();
+                String dataverse = feedInfo.feedConnectionId.getDataverse();
+                String datasetName = feedInfo.feedConnectionId.getDatasetName();
+                String feedName = feedInfo.feedConnectionId.getFeedName();
+                String feedPolicy = feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+                activator.reviveFeed(dataverse, feedName, datasetName, feedPolicy);
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Revived Feed");
+                }
+             
+            }
+        }
+
+        private boolean verfyReasonForFailure(FeedInfo feedInfo) {
+            JobSpecification spec = feedInfo.jobSpec;
+            Set<Constraint> userConstraints = spec.getUserConstraints();
+            List<String> locations = new ArrayList<String>();
+            for (Constraint constraint : userConstraints) {
+                LValueConstraintExpression lexpr = constraint.getLValue();
+                ConstraintExpression cexpr = constraint.getRValue();
+                switch (lexpr.getTag()) {
+                    case PARTITION_LOCATION:
+                        String location = (String) ((ConstantExpression) cexpr).getValue();
+                        locations.add(location);
+                        break;
+                }
+            }
+            Set<String> participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes();
+            List<String> nodesFailedPostSubmission = new ArrayList<String>();
+            for (String location : locations) {
+                if (!participantNodes.contains(location)) {
+                    nodesFailedPostSubmission.add(location);
+                }
+            }
+
+            if (nodesFailedPostSubmission.size() > 0) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Feed failed as nodes failed post submission");
+                }
+                return true;
+            } else {
+                return false;
+            }
+
+        }
+
+        public static class FeedMessengerMessage {
+            private final IFeedMessage message;
+            private final FeedInfo feedInfo;
+
+            public FeedMessengerMessage(IFeedMessage message, FeedInfo feedInfo) {
+                this.message = message;
+                this.feedInfo = feedInfo;
+            }
+
+            public IFeedMessage getMessage() {
+                return message;
+            }
+
+            public FeedInfo getFeedInfo() {
+                return feedInfo;
+            }
+        }
+
+        private static class FeedMessenger implements Runnable {
+
+            private final LinkedBlockingQueue<FeedMessengerMessage> inbox;
+
+            public FeedMessenger(LinkedBlockingQueue<FeedMessengerMessage> inbox) {
+                this.inbox = inbox;
+            }
+
+            public void run() {
+                while (true) {
+                    FeedMessengerMessage message = null;
+                    try {
+                        message = inbox.take();
+                        FeedInfo feedInfo = message.getFeedInfo();
+                        switch (message.getMessage().getMessageType()) {
+                            case SUPER_FEED_MANAGER_ELECT:
+                                Thread.sleep(2000);
+                                sendSuperFeedManangerElectMessage(feedInfo,
+                                        (FeedManagerElectMessage) message.getMessage());
+                                if (LOGGER.isLoggable(Level.WARNING)) {
+                                    LOGGER.warning("Sent super feed manager election message" + message.getMessage());
+                                }
+                        }
+                    } catch (InterruptedException ie) {
+                        break;
+                    }
+                }
+            }
+
+        }
+    }
+
+    public static class FeedInfo {
+        public FeedConnectionId feedConnectionId;
+        public JobSpecification jobSpec;
+        public List<String> ingestLocations = new ArrayList<String>();
+        public List<String> computeLocations = new ArrayList<String>();
+        public List<String> storageLocations = new ArrayList<String>();
+        public JobInfo jobInfo;
+        public Map<String, String> feedPolicy;
+        public JobId jobId;
+
+        public FeedInfo(FeedConnectionId feedId, JobSpecification jobSpec, Map<String, String> feedPolicy, JobId jobId) {
+            this.feedConnectionId = feedId;
+            this.jobSpec = jobSpec;
+            this.feedPolicy = feedPolicy;
+            this.jobId = jobId;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof FeedInfo)) {
+                return false;
+            }
+            return ((FeedInfo) o).feedConnectionId.equals(feedConnectionId);
+        }
+
+        @Override
+        public int hashCode() {
+            return feedConnectionId.hashCode();
+        }
+
+        @Override
+        public String toString() {
+            return feedConnectionId + " job id " + jobId;
+        }
+
+    }
+
+    @Override
+    public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+        Collection<FeedInfo> feedInfos = feedJobNotificationHandler.registeredFeeds.values();
+        FeedFailureReport failureReport = new FeedFailureReport();
+        for (FeedInfo feedInfo : feedInfos) {
+            for (String deadNodeId : deadNodeIds) {
+                if (feedInfo.ingestLocations.contains(deadNodeId)) {
+                    List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+                    if (failures == null) {
+                        failures = new ArrayList<FeedFailure>();
+                        failureReport.failures.put(feedInfo, failures);
+                    }
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Inestion Node Failure! " + deadNodeId);
+                    }
+                    failures.add(new FeedFailure(FeedFailure.FailureType.INGESTION_NODE, deadNodeId));
+                }
+                if (feedInfo.computeLocations.contains(deadNodeId)) {
+                    List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+                    if (failures == null) {
+                        failures = new ArrayList<FeedFailure>();
+                        failureReport.failures.put(feedInfo, failures);
+                    }
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Compute Node Failure! " + deadNodeId);
+                    }
+                    failures.add(new FeedFailure(FeedFailure.FailureType.COMPUTE_NODE, deadNodeId));
+                }
+                if (feedInfo.storageLocations.contains(deadNodeId)) {
+                    List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+                    if (failures == null) {
+                        failures = new ArrayList<FeedFailure>();
+                        failureReport.failures.put(feedInfo, failures);
+                    }
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Storage Node Failure! " + deadNodeId);
+                    }
+                    failures.add(new FeedFailure(FeedFailure.FailureType.STORAGE_NODE, deadNodeId));
+                }
+            }
+        }
+        if (failureReport.failures.isEmpty()) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                StringBuilder builder = new StringBuilder();
+                builder.append("No feed is affected by the failure of node(s): ");
+                for (String deadNodeId : deadNodeIds) {
+                    builder.append(deadNodeId + " ");
+                }
+                LOGGER.info(builder.toString());
+            }
+            return new HashSet<IClusterManagementWork>();
+        } else {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                StringBuilder builder = new StringBuilder();
+                builder.append("Feed affected by the failure of node(s): ");
+                for (String deadNodeId : deadNodeIds) {
+                    builder.append(deadNodeId + " ");
+                }
+                builder.append("\n");
+                for (FeedInfo fInfo : failureReport.failures.keySet()) {
+                    builder.append(fInfo.feedConnectionId);
+                    feedJobNotificationHandler.deregisterFeed(fInfo);
+                }
+                LOGGER.warning(builder.toString());
+            }
+            return handleFailure(failureReport);
+        }
+    }
+
+    private Set<IClusterManagementWork> handleFailure(FeedFailureReport failureReport) {
+        reportFeedFailure(failureReport);
+        Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+        Map<String, Map<FeedInfo, List<FailureType>>> failureMap = new HashMap<String, Map<FeedInfo, List<FailureType>>>();
+        FeedPolicyAccessor fpa = null;
+        List<FeedInfo> feedsToTerminate = new ArrayList<FeedInfo>();
+        for (Map.Entry<FeedInfo, List<FeedFailure>> entry : failureReport.failures.entrySet()) {
+            FeedInfo feedInfo = entry.getKey();
+            fpa = new FeedPolicyAccessor(feedInfo.feedPolicy);
+            if (!fpa.continueOnHardwareFailure()) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    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");
+                }
+                continue;
+            } else {
+                // insert feed recovery mode 
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    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");
+                }
+            }
+
+            List<FeedFailure> feedFailures = entry.getValue();
+            boolean recoveryPossible = true;
+            for (FeedFailure feedFailure : feedFailures) {
+                switch (feedFailure.failureType) {
+                    case COMPUTE_NODE:
+                    case INGESTION_NODE:
+                        Map<FeedInfo, List<FailureType>> failuresBecauseOfThisNode = failureMap.get(feedFailure.nodeId);
+                        if (failuresBecauseOfThisNode == null) {
+                            failuresBecauseOfThisNode = new HashMap<FeedInfo, List<FailureType>>();
+                            failuresBecauseOfThisNode.put(feedInfo, new ArrayList<FailureType>());
+                            failureMap.put(feedFailure.nodeId, failuresBecauseOfThisNode);
+                        }
+                        List<FailureType> feedF = failuresBecauseOfThisNode.get(feedInfo);
+                        if (feedF == null) {
+                            feedF = new ArrayList<FailureType>();
+                            failuresBecauseOfThisNode.put(feedInfo, feedF);
+                        }
+                        feedF.add(feedFailure.failureType);
+                        break;
+                    case STORAGE_NODE:
+                        recoveryPossible = false;
+                        if (LOGGER.isLoggable(Level.SEVERE)) {
+                            LOGGER.severe("Unrecoverable situation! lost storage node for the feed "
+                                    + feedInfo.feedConnectionId);
+                        }
+                        List<String> requiredNodeIds = dependentFeeds.get(feedInfo);
+                        if (requiredNodeIds == null) {
+                            requiredNodeIds = new ArrayList<String>();
+                            dependentFeeds.put(feedInfo, requiredNodeIds);
+                        }
+                        requiredNodeIds.add(feedFailure.nodeId);
+                        failuresBecauseOfThisNode = failureMap.get(feedFailure.nodeId);
+                        if (failuresBecauseOfThisNode != null) {
+                            failuresBecauseOfThisNode.remove(feedInfo);
+                            if (failuresBecauseOfThisNode.isEmpty()) {
+                                failureMap.remove(feedFailure.nodeId);
+                            }
+                        }
+                        feedsToTerminate.add(feedInfo);
+                        break;
+                }
+            }
+            if (!recoveryPossible) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Terminating irrecoverable feed (loss of storage node) ");
+                }
+            }
+        }
+
+        if (!feedsToTerminate.isEmpty()) {
+            Thread t = new Thread(new FeedsDeActivator(feedsToTerminate));
+            t.start();
+        }
+
+        int numRequiredNodes = 0;
+        for (Entry<String, Map<FeedInfo, List<FeedFailure.FailureType>>> entry : failureMap.entrySet()) {
+            Map<FeedInfo, List<FeedFailure.FailureType>> v = entry.getValue();
+            for (FeedInfo finfo : feedsToTerminate) {
+                v.remove(finfo);
+            }
+            if (v.size() > 0) {
+                numRequiredNodes++;
+            }
+        }
+
+        if (numRequiredNodes > 0) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Number of additional nodes requested " + numRequiredNodes);
+            }
+            AddNodeWork addNodesWork = new AddNodeWork(failureMap.keySet().size(), this);
+            work.add(addNodesWork);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                Map<FeedInfo, List<FeedFailure>> feedFailures = failureReport.failures;
+                for (Entry<FeedInfo, List<FeedFailure>> entry : feedFailures.entrySet()) {
+                    for (FeedFailure f : entry.getValue()) {
+                        LOGGER.info("Feed Failure! " + f.failureType + " " + f.nodeId);
+                    }
+                }
+            }
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Registered work id: " + addNodesWork.getWorkId());
+            }
+            feedWorkRequestResponseHandler.registerFeedWork(addNodesWork.getWorkId(), failureReport);
+        } else {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Not requesting any new node. Feeds unrecoverable until the lost node(s) rejoin");
+            }
+        }
+        return work;
+    }
+
+    private void reportFeedFailure(FeedFailureReport failureReport) {
+        MetadataTransactionContext ctx = null;
+        FeedActivity fa = null;
+        Map<String, String> feedActivityDetails = new HashMap<String, String>();
+        StringBuilder builder = new StringBuilder();
+        MetadataManager.INSTANCE.acquireWriteLatch();
+        try {
+            ctx = MetadataManager.INSTANCE.beginTransaction();
+            for (Entry<FeedInfo, List<FeedFailure>> entry : failureReport.failures.entrySet()) {
+                FeedInfo feedInfo = entry.getKey();
+                List<FeedFailure> feedFailures = entry.getValue();
+                for (FeedFailure failure : feedFailures) {
+                    builder.append(failure + ",");
+                }
+                builder.deleteCharAt(builder.length() - 1);
+                feedActivityDetails.put(FeedActivityDetails.FEED_NODE_FAILURE, builder.toString());
+                fa = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+                        feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+                        FeedActivityType.FEED_FAILURE, feedActivityDetails);
+                MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedConnectionId, fa);
+            }
+            MetadataManager.INSTANCE.commitTransaction(ctx);
+        } catch (Exception e) {
+            if (ctx != null) {
+                try {
+                    MetadataManager.INSTANCE.abortTransaction(ctx);
+                } catch (Exception e2) {
+                    e2.addSuppressed(e);
+                    throw new IllegalStateException("Unable to abort transaction " + e2);
+                }
+            }
+        } finally {
+            MetadataManager.INSTANCE.releaseWriteLatch();
+        }
+    }
+
+    private static void sendSuperFeedManangerElectMessage(FeedInfo feedInfo, FeedManagerElectMessage electMessage) {
+        try {
+            Dataverse dataverse = new Dataverse(feedInfo.feedConnectionId.getDataverse(),
+                    NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, 0);
+            AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse);
+            JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+
+            IOperatorDescriptor feedMessenger;
+            AlgebricksPartitionConstraint messengerPc;
+            Set<String> locations = new HashSet<String>();
+            locations.addAll(feedInfo.computeLocations);
+            locations.addAll(feedInfo.ingestLocations);
+            locations.addAll(feedInfo.storageLocations);
+
+            Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildSendFeedMessageRuntime(
+                    spec, dataverse.getDataverseName(), feedInfo.feedConnectionId.getFeedName(),
+                    feedInfo.feedConnectionId.getDatasetName(), electMessage, locations.toArray(new String[] {}));
+            feedMessenger = p.first;
+            messengerPc = p.second;
+            AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedMessenger, messengerPc);
+
+            NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+            AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
+            spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
+            spec.addRoot(nullSink);
+
+            JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(spec);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(" Super Feed Manager Message: " + electMessage + " Job Id " + jobId);
+            }
+
+        } catch (Exception e) {
+            e.printStackTrace();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Exception in sending super feed manager elect message: " + feedInfo.feedConnectionId + " "
+                        + e.getMessage());
+            }
+        }
+    }
+
+    public static class FeedFailure {
+
+        public enum FailureType {
+            INGESTION_NODE,
+            COMPUTE_NODE,
+            STORAGE_NODE
+        }
+
+        public FailureType failureType;
+        public String nodeId;
+
+        public FeedFailure(FailureType failureType, String nodeId) {
+            this.failureType = failureType;
+            this.nodeId = nodeId;
+        }
+
+        @Override
+        public String toString() {
+            return failureType + " (" + nodeId + ") ";
+        }
+    }
+
+    @Override
+    public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
+        State newState = AsterixClusterProperties.INSTANCE.getState();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(joinedNodeId + " joined the cluster. " + "Asterix state: " + newState);
+        }
+
+        boolean needToReActivateFeeds = !newState.equals(state) && (newState == State.ACTIVE);
+        if (needToReActivateFeeds) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(joinedNodeId + " Resuming loser feeds (if any)");
+            }
+            try {
+                FeedsActivator activator = new FeedsActivator();
+                (new Thread(activator)).start();
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Exception in resuming feeds" + e.getMessage());
+                }
+            }
+            state = newState;
+        } else {
+            List<FeedInfo> feedsThatCanBeRevived = new ArrayList<FeedInfo>();
+            for (Entry<FeedInfo, List<String>> entry : dependentFeeds.entrySet()) {
+                List<String> requiredNodeIds = entry.getValue();
+                if (requiredNodeIds.contains(joinedNodeId)) {
+                    requiredNodeIds.remove(joinedNodeId);
+                    if (requiredNodeIds.isEmpty()) {
+                        feedsThatCanBeRevived.add(entry.getKey());
+                    }
+                }
+            }
+            if (!feedsThatCanBeRevived.isEmpty()) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(joinedNodeId + " Resuming feeds after rejoining of node " + joinedNodeId);
+                }
+                FeedsActivator activator = new FeedsActivator(feedsThatCanBeRevived);
+                (new Thread(activator)).start();
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
+        try {
+            responseInbox.put(response);
+        } catch (InterruptedException e) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Interrupted exception");
+            }
+        }
+    }
+
+    @Override
+    public void notifyStateChange(State previousState, State newState) {
+        switch (newState) {
+            case ACTIVE:
+                if (previousState.equals(State.UNUSABLE)) {
+                    try {
+                        FeedsActivator activator = new FeedsActivator();
+                        (new Thread(activator)).start();
+                    } catch (Exception e) {
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Exception in resuming feeds" + e.getMessage());
+                        }
+                    }
+                }
+                break;
+        }
+
+    }
+
+    private static class FeedsActivator implements Runnable {
+
+        private List<FeedInfo> feedsToRevive;
+        private Mode mode;
+
+        public enum Mode {
+            REVIVAL_POST_CLUSTER_REBOOT,
+            REVIVAL_POST_NODE_REJOIN
+        }
+
+        public FeedsActivator() {
+            this.mode = Mode.REVIVAL_POST_CLUSTER_REBOOT;
+        }
+
+        public FeedsActivator(List<FeedInfo> feedsToRevive) {
+            this.feedsToRevive = feedsToRevive;
+            this.mode = Mode.REVIVAL_POST_NODE_REJOIN;
+        }
+
+        @Override
+        public void run() {
+            switch (mode) {
+                case REVIVAL_POST_CLUSTER_REBOOT:
+                    revivePostClusterReboot();
+                    break;
+                case REVIVAL_POST_NODE_REJOIN:
+                    try {
+                        Thread.sleep(10000);
+                    } catch (InterruptedException e1) {
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Attempt to resume feed interrupted");
+                        }
+                        throw new IllegalStateException(e1.getMessage());
+                    }
+                    for (FeedInfo finfo : feedsToRevive) {
+                        try {
+                            JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(finfo.jobSpec);
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                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.feedConnectionId + " " + e.getMessage());
+                            }
+                        }
+                    }
+            }
+        }
+
+        private void revivePostClusterReboot() {
+            MetadataTransactionContext ctx = null;
+
+            try {
+
+                Thread.sleep(4000);
+                MetadataManager.INSTANCE.init();
+                ctx = MetadataManager.INSTANCE.beginTransaction();
+                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());
+                    for (FeedActivity fa : activeFeeds) {
+                        LOGGER.info("Active feed " + fa.getDataverseName() + ":" + fa.getDatasetName());
+                    }
+                }
+                for (FeedActivity fa : activeFeeds) {
+                    String feedPolicy = fa.getFeedActivityDetails().get(FeedActivityDetails.FEED_POLICY_NAME);
+                    FeedPolicy policy = MetadataManager.INSTANCE.getFeedPolicy(ctx, fa.getDataverseName(), feedPolicy);
+                    if (policy == null) {
+                        policy = MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                                feedPolicy);
+                        if (policy == null) {
+                            if (LOGGER.isLoggable(Level.SEVERE)) {
+                                LOGGER.severe("Unable to resume feed: " + fa.getDataverseName() + ":"
+                                        + fa.getDatasetName() + "." + " Unknown policy :" + feedPolicy);
+                            }
+                            continue;
+                        }
+                    }
+
+                    FeedPolicyAccessor fpa = new FeedPolicyAccessor(policy.getProperties());
+                    if (fpa.autoRestartOnClusterReboot()) {
+                        String dataverse = fa.getDataverseName();
+                        String datasetName = fa.getDatasetName();
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Resuming feed after cluster revival: " + dataverse + ":" + datasetName
+                                    + " using policy " + feedPolicy);
+                        }
+                        reviveFeed(dataverse, fa.getFeedName(), datasetName, feedPolicy);
+                    } else {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Feed " + fa.getDataverseName() + ":" + fa.getDatasetName()
+                                    + " governed by policy" + feedPolicy
+                                    + " does not state auto restart after cluster revival");
+                        }
+                    }
+                }
+                MetadataManager.INSTANCE.commitTransaction(ctx);
+
+            } catch (Exception e) {
+                e.printStackTrace();
+                try {
+                    MetadataManager.INSTANCE.abortTransaction(ctx);
+                } catch (Exception e1) {
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Exception in aborting" + e.getMessage());
+                    }
+                    throw new IllegalStateException(e1);
+                }
+            }
+        }
+
+        public 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));
+                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);
+                AqlTranslator translator = new AqlTranslator(statements, writer, pc, DisplayFormat.TEXT);
+                translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, false);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Resumed feed: " + dataverse + ":" + dataset + " using policy " + feedPolicy);
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Exception in resuming loser feed: " + dataverse + ":" + dataset + " using policy "
+                            + feedPolicy + " Exception " + e.getMessage());
+                }
+            }
+        }
+    }
+
+    public static class FeedsDeActivator implements Runnable {
+
+        private List<FeedInfo> feedsToTerminate;
+
+        public FeedsDeActivator(List<FeedInfo> feedsToTerminate) {
+            this.feedsToTerminate = feedsToTerminate;
+        }
+
+        @Override
+        public void run() {
+            for (FeedInfo feedInfo : feedsToTerminate) {
+                endFeed(feedInfo);
+            }
+        }
+
+        private void endFeed(FeedInfo feedInfo) {
+            MetadataTransactionContext ctx = null;
+            PrintWriter writer = new PrintWriter(System.out, true);
+            SessionConfig pc = new SessionConfig(true, false, false, false, false, false, true, true, false);
+            try {
+                ctx = MetadataManager.INSTANCE.beginTransaction();
+                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.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.feedConnectionId);
+                }
+                MetadataManager.INSTANCE.commitTransaction(ctx);
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Exception in ending loser feed: " + feedInfo.feedConnectionId + " Exception "
+                            + e.getMessage());
+                }
+                e.printStackTrace();
+                try {
+                    MetadataManager.INSTANCE.abortTransaction(ctx);
+                } catch (Exception e2) {
+                    e2.addSuppressed(e);
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Exception in aborting transaction! System is in inconsistent state");
+                    }
+                }
+
+            }
+
+        }
+    }
+}
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
new file mode 100644
index 0000000..3197bbb
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
@@ -0,0 +1,340 @@
+/*
+ * 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.hyracks.bootstrap;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailure;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailureReport;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedInfo;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedsDeActivator;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression.ExpressionTag;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedWorkRequestResponseHandler implements Runnable {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedWorkRequestResponseHandler.class.getName());
+
+    private final LinkedBlockingQueue<IClusterManagementWorkResponse> inbox;
+
+    private Map<Integer, FeedFailureReport> feedsWaitingForResponse = new HashMap<Integer, FeedFailureReport>();
+
+    public FeedWorkRequestResponseHandler(LinkedBlockingQueue<IClusterManagementWorkResponse> inbox) {
+        this.inbox = inbox;
+    }
+
+    @Override
+    public void run() {
+        while (true) {
+            IClusterManagementWorkResponse response = null;
+            try {
+                response = inbox.take();
+            } catch (InterruptedException e) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Interrupted exception " + e.getMessage());
+                }
+            }
+            IClusterManagementWork submittedWork = response.getWork();
+            switch (submittedWork.getClusterManagementWorkType()) {
+                case ADD_NODE:
+                    AddNodeWorkResponse resp = (AddNodeWorkResponse) response;
+                    switch (resp.getStatus()) {
+                        case FAILURE:
+                            if (LOGGER.isLoggable(Level.WARNING)) {
+                                LOGGER.warning("Request " + resp.getWork() + " not completed");
+                            }
+                            break;
+                        case SUCCESS:
+                            if (LOGGER.isLoggable(Level.WARNING)) {
+                                LOGGER.warning("Request " + resp.getWork() + " completed");
+                            }
+                            break;
+                    }
+
+                    AddNodeWork work = (AddNodeWork) submittedWork;
+                    FeedFailureReport failureReport = feedsWaitingForResponse.remove(work.getWorkId());
+                    Set<FeedInfo> affectedFeeds = failureReport.failures.keySet();
+                    for (FeedInfo feedInfo : affectedFeeds) {
+                        try {
+                            recoverFeed(feedInfo, work, resp, failureReport.failures.get(feedInfo));
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info("Recovered feed:" + feedInfo);
+                            }
+                        } catch (Exception e) {
+                            if (LOGGER.isLoggable(Level.SEVERE)) {
+                                LOGGER.severe("Unable to recover feed:" + feedInfo);
+                            }
+                        }
+                    }
+                    break;
+                case REMOVE_NODE:
+                    break;
+            }
+        }
+    }
+
+    private void recoverFeed(FeedInfo feedInfo, AddNodeWork work, AddNodeWorkResponse resp,
+            List<FeedFailure> feedFailures) throws Exception {
+        List<String> failedNodeIds = new ArrayList<String>();
+        for (FeedFailure feedFailure : feedFailures) {
+            failedNodeIds.add(feedFailure.nodeId);
+        }
+        List<String> chosenReplacements = new ArrayList<String>();
+        String metadataNodeName = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
+        chosenReplacements.add(metadataNodeName);
+        switch (resp.getStatus()) {
+            case FAILURE:
+                for (FeedFailure feedFailure : feedFailures) {
+                    switch (feedFailure.failureType) {
+                        case INGESTION_NODE:
+                            String replacement = getInternalReplacement(feedInfo, feedFailure, failedNodeIds,
+                                    chosenReplacements);
+                            chosenReplacements.add(replacement);
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info("Existing  node:" + replacement + " chosen to replace "
+                                        + feedFailure.nodeId);
+                            }
+                            alterFeedJobSpec(feedInfo, resp, feedFailure.nodeId, replacement);
+                            break;
+                    }
+                }
+                break;
+            case SUCCESS:
+                List<String> nodesAdded = resp.getNodesAdded();
+                int numNodesAdded = nodesAdded.size();
+                int nodeIndex = 0;
+                for (FeedFailure feedFailure : feedFailures) {
+                    switch (feedFailure.failureType) {
+                        case INGESTION_NODE:
+                            String replacement = null;
+                            if (nodeIndex <= numNodesAdded - 1) {
+                                replacement = nodesAdded.get(nodeIndex);
+                                if (LOGGER.isLoggable(Level.INFO)) {
+                                    LOGGER.info("Newly added node:" + replacement + " chosen to replace "
+                                            + feedFailure.nodeId);
+                                }
+                            } else {
+                                replacement = getInternalReplacement(feedInfo, feedFailure, failedNodeIds,
+                                        chosenReplacements);
+                                if (LOGGER.isLoggable(Level.INFO)) {
+                                    LOGGER.info("Existing node:" + replacement + " chosen to replace "
+                                            + feedFailure.nodeId);
+                                }
+                                chosenReplacements.add(replacement);
+                            }
+                            alterFeedJobSpec(feedInfo, resp, feedFailure.nodeId, replacement);
+                            nodeIndex++;
+                            break;
+                        default: // ingestion nodes and compute nodes (in currrent implementation) coincide.
+                                 // so correcting ingestion node failure also takes care of compute nodes failure. 
+                                 // Storage node failures cannot be recovered from as in current implementation, we 
+                                 // do not have data replication.
+                    }
+                }
+                break;
+        }
+
+        JobSpecification spec = feedInfo.jobSpec;
+        System.out.println("Final recovery Job Spec \n" + spec);
+        Thread.sleep(5000);
+        AsterixAppContextInfo.getInstance().getHcc().startJob(feedInfo.jobSpec);
+    }
+
+    private String getInternalReplacement(FeedInfo feedInfo, FeedFailure feedFailure, List<String> failedNodeIds,
+            List<String> chosenReplacements) {
+        String failedNodeId = feedFailure.nodeId;
+        String replacement = null;;
+        // TODO 1st preference is given to any other participant node that is not involved in the feed.
+        //      2nd preference is given to a compute node.
+        //      3rd preference is given to a storage node
+        Set<String> participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes();
+        if (participantNodes != null && !participantNodes.isEmpty()) {
+            List<String> pNodesClone = new ArrayList<String>();
+            pNodesClone.addAll(participantNodes);
+            pNodesClone.removeAll(feedInfo.storageLocations);
+            pNodesClone.removeAll(feedInfo.computeLocations);
+            pNodesClone.removeAll(feedInfo.ingestLocations);
+            pNodesClone.removeAll(chosenReplacements);
+
+            if (LOGGER.isLoggable(Level.INFO)) {
+                for (String candidateNode : pNodesClone) {
+                    LOGGER.info("Candidate for replacement:" + candidateNode);
+                }
+            }
+            if (!pNodesClone.isEmpty()) {
+                String[] participantNodesArray = pNodesClone.toArray(new String[] {});
+
+                replacement = participantNodesArray[new Random().nextInt(participantNodesArray.length)];
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Participant Node: " + replacement + " chosen as replacement for " + failedNodeId);
+                }
+            }
+        }
+
+        if (replacement == null) {
+            feedInfo.computeLocations.removeAll(failedNodeIds);
+            boolean computeNodeSubstitute = (feedInfo.computeLocations.size() > 1);
+            if (computeNodeSubstitute) {
+                replacement = feedInfo.computeLocations.get(0);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Compute node:" + replacement + " chosen to replace " + failedNodeId);
+                }
+            } else {
+                replacement = feedInfo.storageLocations.get(0);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Storage node:" + replacement + " chosen to replace " + failedNodeId);
+                }
+            }
+        }
+        return replacement;
+    }
+
+    private void alterFeedJobSpec(FeedInfo feedInfo, AddNodeWorkResponse resp, String failedNodeId, String replacement) {
+        if (replacement == null) {
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.severe("Unable to find replacement for failed node :" + failedNodeId);
+                LOGGER.severe("Feed: " + feedInfo.feedConnectionId + " will be terminated");
+            }
+            List<FeedInfo> feedsToTerminate = new ArrayList<FeedInfo>();
+            feedsToTerminate.add(feedInfo);
+            Thread t = new Thread(new FeedsDeActivator(feedsToTerminate));
+            t.start();
+        } else {
+            replaceNode(feedInfo.jobSpec, failedNodeId, replacement);
+        }
+    }
+
+    private void replaceNode(JobSpecification jobSpec, String failedNodeId, String replacementNode) {
+        Set<Constraint> userConstraints = jobSpec.getUserConstraints();
+        List<Constraint> locationConstraintsToReplace = new ArrayList<Constraint>();
+        List<Constraint> countConstraintsToReplace = new ArrayList<Constraint>();
+        List<OperatorDescriptorId> modifiedOperators = new ArrayList<OperatorDescriptorId>();
+        Map<OperatorDescriptorId, List<Constraint>> candidateConstraints = new HashMap<OperatorDescriptorId, List<Constraint>>();
+        Map<OperatorDescriptorId, Map<Integer, String>> newConstraints = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
+        OperatorDescriptorId opId = null;
+        for (Constraint constraint : userConstraints) {
+            LValueConstraintExpression lexpr = constraint.getLValue();
+            ConstraintExpression cexpr = constraint.getRValue();
+            switch (lexpr.getTag()) {
+                case PARTITION_COUNT:
+                    opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+                    if (modifiedOperators.contains(opId)) {
+                        countConstraintsToReplace.add(constraint);
+                    } else {
+                        List<Constraint> clist = candidateConstraints.get(opId);
+                        if (clist == null) {
+                            clist = new ArrayList<Constraint>();
+                            candidateConstraints.put(opId, clist);
+                        }
+                        clist.add(constraint);
+                    }
+                    break;
+                case PARTITION_LOCATION:
+                    opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+                    String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
+                    if (oldLocation.equals(failedNodeId)) {
+                        locationConstraintsToReplace.add(constraint);
+                        modifiedOperators.add(((PartitionLocationExpression) lexpr).getOperatorDescriptorId());
+                        Map<Integer, String> newLocs = newConstraints.get(opId);
+                        if (newLocs == null) {
+                            newLocs = new HashMap<Integer, String>();
+                            newConstraints.put(opId, newLocs);
+                        }
+                        int partition = ((PartitionLocationExpression) lexpr).getPartition();
+                        newLocs.put(partition, replacementNode);
+                    } else {
+                        if (modifiedOperators.contains(opId)) {
+                            locationConstraintsToReplace.add(constraint);
+                            Map<Integer, String> newLocs = newConstraints.get(opId);
+                            if (newLocs == null) {
+                                newLocs = new HashMap<Integer, String>();
+                                newConstraints.put(opId, newLocs);
+                            }
+                            int partition = ((PartitionLocationExpression) lexpr).getPartition();
+                            newLocs.put(partition, oldLocation);
+                        } else {
+                            List<Constraint> clist = candidateConstraints.get(opId);
+                            if (clist == null) {
+                                clist = new ArrayList<Constraint>();
+                                candidateConstraints.put(opId, clist);
+                            }
+                            clist.add(constraint);
+                        }
+                    }
+                    break;
+            }
+        }
+
+        jobSpec.getUserConstraints().removeAll(locationConstraintsToReplace);
+        jobSpec.getUserConstraints().removeAll(countConstraintsToReplace);
+
+        for (OperatorDescriptorId mopId : modifiedOperators) {
+            List<Constraint> clist = candidateConstraints.get(mopId);
+            if (clist != null && !clist.isEmpty()) {
+                jobSpec.getUserConstraints().removeAll(clist);
+
+                for (Constraint c : clist) {
+                    if (c.getLValue().getTag().equals(ExpressionTag.PARTITION_LOCATION)) {
+                        ConstraintExpression cexpr = c.getRValue();
+                        int partition = ((PartitionLocationExpression) c.getLValue()).getPartition();
+                        String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
+                        newConstraints.get(mopId).put(partition, oldLocation);
+                    }
+                }
+            }
+        }
+
+        for (Entry<OperatorDescriptorId, Map<Integer, String>> entry : newConstraints.entrySet()) {
+            OperatorDescriptorId nopId = entry.getKey();
+            Map<Integer, String> clist = entry.getValue();
+            IOperatorDescriptor op = jobSpec.getOperatorMap().get(nopId);
+            String[] locations = new String[clist.size()];
+            for (int i = 0; i < locations.length; i++) {
+                locations[i] = clist.get(i);
+            }
+            PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, op, locations);
+        }
+
+    }
+
+    public void registerFeedWork(int workId, FeedFailureReport failureReport) {
+        feedsWaitingForResponse.put(workId, failureReport);
+    }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 96d0617..187394a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -25,14 +25,18 @@
 import edu.uci.ics.asterix.common.api.AsterixThreadFactory;
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.SystemState;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataNode;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataBootstrap;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
@@ -59,14 +63,22 @@
         JVMShutdownHook sHook = new JVMShutdownHook(this);
         Runtime.getRuntime().addShutdownHook(sHook);
 
-     
         runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
+        AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
+                .getMetadataProperties();
+        if (!metadataProperties.getNodeNames().contains(ncApplicationContext.getNodeId())) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Substitute node joining : " + ncApplicationContext.getNodeId());
+            }
+            updateOnNodeJoin();
+        }
         runtimeContext.initialize();
         ncApplicationContext.setApplicationObject(runtimeContext);
 
         // #. recover if the system is corrupted by checking system state.
         IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
         systemState = recoveryMgr.getSystemState();
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("System is in a state: " + systemState);
         }
@@ -130,21 +142,25 @@
         isMetadataNode = nodeId.equals(metadataProperties.getMetadataNodeName());
         if (isMetadataNode) {
             registerRemoteMetadataNode(proxy);
+        }
+        MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
+        MetadataManager.INSTANCE.init();
 
+        if (isMetadataNode) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Bootstrapping metadata");
             }
-            MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
-            MetadataManager.INSTANCE.init();
+
             MetadataBootstrap.startUniverse(((IAsterixPropertiesProvider) runtimeContext), ncApplicationContext,
                     systemState == SystemState.NEW_UNIVERSE);
             MetadataBootstrap.startDDLRecovery();
         }
 
+        ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting lifecycle components");
         }
-        
+
         Map<String, String> lifecycleMgmtConfiguration = new HashMap<String, String>();
         String key = LifeCycleComponentManager.Config.DUMP_PATH_KEY;
         String value = metadataProperties.getCoredumpPath(nodeId);
@@ -156,7 +172,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Configured:" + LifeCycleComponentManager.INSTANCE);
         }
-        
+
         LifeCycleComponentManager.INSTANCE.startAll();
 
         IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
@@ -177,6 +193,47 @@
         }
     }
 
+    private void updateOnNodeJoin() {
+        AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
+                .getMetadataProperties();
+        AsterixTransactionProperties txnProperties = ((IAsterixPropertiesProvider) runtimeContext)
+                .getTransactionProperties();
+        if (!metadataProperties.getNodeNames().contains(nodeId)) {
+            metadataProperties.getNodeNames().add(nodeId);
+            Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+            String asterixInstanceName = cluster.getInstanceName();
+            Node self = null;
+            for (Node node : cluster.getSubstituteNodes().getNode()) {
+                String ncId = asterixInstanceName + "_" + node.getId();
+                if (ncId.equalsIgnoreCase(nodeId)) {
+                    String storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+                    metadataProperties.getStores().put(nodeId, storeDir.split(","));
+
+                    String coredumpPath = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+                    metadataProperties.getCoredumpPaths().put(nodeId, coredumpPath);
+
+                    String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+                    txnProperties.getLogDirectories().put(nodeId, txnLogDir);
+
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Store set to : " + storeDir);
+                        LOGGER.info("Coredump dir set to : " + coredumpPath);
+                        LOGGER.info("Transaction log dir set to :" + txnLogDir);
+                    }
+                    self = node;
+                    break;
+                }
+            }
+            if (self != null) {
+                cluster.getSubstituteNodes().getNode().remove(self);
+                cluster.getNode().add(self);
+            } else {
+                throw new IllegalStateException("Unknown node joining the cluster");
+            }
+        }
+
+    }
+
     /**
      * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop} method.
      */
@@ -202,4 +259,4 @@
         }
     }
 
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/resources/feed/dashboard.html b/asterix-app/src/main/resources/feed/dashboard.html
new file mode 100644
index 0000000..805f8ea
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/dashboard.html
@@ -0,0 +1,127 @@
+<html>
+  <head>
+    <script type="text/javascript" src="http://ajax.googleapis.com/ajax/libs/jquery/1.6.2/jquery.min.js"></script>
+    <script type="text/javascript" src="/webui/static/js/smoothie.js"></script>
+    <script type="text/javascript">
+    $(document).ready(function() {
+        var feedSeries = new TimeSeries();
+      	var state = "ACTIVE";
+        var dataverse = "%s";
+        var dataset = "%s";
+        var feed = "%s";
+        var ingestLocations = "%s";
+        var computeLocations = "%s";
+        var storageLocations = "%s";
+        var ingestionPolicy = "%s";
+        var activeSince = "%s";
+        var targetUrl = "/feed/data?dataverse=" + dataverse + "&dataset=" + dataset + "&feed=" + feed;
+
+
+        var ingestionNodes = ingestLocations.split(",");
+        var numIngestionNodes = ingestionNodes.length;
+        var seriesOptions = { strokeStyle: 'rgba(0, 255, 0, 1)', fillStyle: 'rgba(0, 255, 0, 0.2)', lineWidth: 4 };
+        var ingestionTimeSeries = new Array();
+        var graphNames = new Array();
+
+        $.ajaxSetup({ cache: false });
+        setInterval(fetchFeedReport, 5000);
+        function fetchFeedReport() {
+          $.ajax({
+            url: '/feed/data?dataverse=' + dataverse + '&dataset=' + dataset + '&feed=' + feed,
+            method: 'GET',
+            dataType: 'json',
+            success: onFeedReportReceived
+            });
+         }
+
+         
+        function onFeedReportReceived(data) {
+            var status = data["status"];
+            if(status == ("ended")){
+              ingestLocations = " ";
+              computeLocations = " ";
+              storageLocations = " ";
+              ingestionPolicy = " ";
+              state = "INACTIVE";
+              document.location.reload(true);
+            } else {
+            	var type = data["type"];
+            	if (type == ("reload")) {
+              		ingestLocations  = data["ingestLocations"];
+              		computeLocations = data["computeLocations"];
+              		storageLocations = data["storageLocations"];
+              		document.location.reload(true);
+           		 } else {
+            		var report = data["value"];
+            		var tputArray = report.split("|");
+            		var covered = 0;
+            		var totalTput = 0;
+            		for( var i = 0; i < tputArray.length-1; i ++){
+               			ingestionTimeSeries[i].append(data["time"], tputArray[i]);
+               			covered++;
+               			totalTput += parseInt(tputArray[i]);
+            		}          
+            		for( var j = covered; j < numIngestionNodes; j++){
+                		ingestionTimeSeries[j].append(data["time"], 0);
+            		}          
+            		ingestionTimeSeries[numIngestionNodes].append(data["time"], totalTput);
+            	}
+            }
+        }
+
+        function myYRangeFunction(range) {
+           var min = 0;
+           var max = 5000;
+           return {min: min, max: max};
+        }
+
+        function initTimeline(ingestLocations) {
+
+          document.write("<i>" + "Feed Ingestion" + " "  + "<i>");
+          document.write("<br />" + "Ingestion Locations: " + ingestLocations.replace(",",", "));
+          document.write("<br />" + "Compute Locations: " + computeLocations.replace(",",", "));
+          document.write("<br />" + "Storage Locations: " + storageLocations.replace(",",", "));
+          document.write("<br />" + "Ingestion Policy: " + ingestionPolicy);
+          document.write("<br />" + "Status: " + state);
+          document.write("<br />");
+          document.write("<br />");
+    
+          for( var i = 0; i < numIngestionNodes; i++){
+              graphNames[i] = "Partition " + i;
+          }          
+
+          if(numIngestionNodes > 1){
+             graphNames[numIngestionNodes] = "IngestionThroughput";
+             drawCanvas(graphNames[numIngestionNodes]);
+             ingestionTimeSeries[numIngestionNodes] = new TimeSeries();
+             drawChart(graphNames[numIngestionNodes], ingestionTimeSeries[numIngestionNodes]);
+          }
+
+          for( var j = 0; j < numIngestionNodes; j++){
+             drawCanvas(graphNames[j]);
+             ingestionTimeSeries[j] = new TimeSeries();
+             drawChart(graphNames[j], ingestionTimeSeries[j]);
+          }
+        }
+
+        function drawCanvas(chartName) {
+          document.write("<br />");
+          document.write("<br />");
+          document.write("<i>" + chartName + "</i>");
+          document.write("<br />");
+          document.write("<canvas id="+ "\"" + chartName + "\"" + " " + "width=\"500\" height=\"250\"></canvas>");
+        }   
+
+        function drawChart(chartName, ingestionTimeSeries) {
+             var ingestionChart = new SmoothieChart({ timestampFormatter:SmoothieChart.timeFormatter, interpolation:'linear', minValue:0, millisPerPixel: 20, grid: { strokeStyle: '#555555', lineWidth: 1, millisPerLine: 1000, verticalSections: 10 }});
+             ingestionChart.addTimeSeries(ingestionTimeSeries, seriesOptions);
+             ingestionChart.streamTo(document.getElementById(chartName, 500));
+        }
+
+        initTimeline(ingestLocations);
+     });   
+    </script>
+  </head>
+  <body></body>
+</html>
+
diff --git a/asterix-app/src/main/resources/feed/home.html b/asterix-app/src/main/resources/feed/home.html
new file mode 100644
index 0000000..5b1721b
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/home.html
@@ -0,0 +1,82 @@
+<!--
+ ! 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.
+ !-->
+<!DOCTYPE html>
+<html lang="en">
+<head>
+<meta name="description" content="ASTERIX WEB PAGE" />
+<meta name="viewport" content="width=device-width, initial-scale=1.0">
+<link href='http://fonts.googleapis.com/css?family=Bitter|PT+Sans+Caption|Open+Sans' rel='stylesheet' type='text/css'>
+<script src="/webui/static/js/jquery.min.js"></script>
+
+<link href="/webui/static/css/bootstrap.min.css" rel="stylesheet" type="text/css" />
+<link href="/webui/static/css/bootstrap-responsive.min.css" rel="stylesheet" type="text/css" />
+
+<script src="/webui/static/js/bootstrap.min.js"></script>
+
+<link href="/webui/static/css/style.css" rel="stylesheet" type="text/css" />
+
+
+<meta charset=utf-8 />
+<title>AsterixDB Web Interface</title>
+</head>
+
+<body>
+  <div class="navbar navbar-fixed-top">
+    <div class="navbar-inner">
+      <div class="container">
+        <a class="btn btn-navbar" data-toggle="collapse" data-target=".nav-collapse">
+          <span class="icon-bar"></span>
+          <span class="icon-bar"></span>
+          <span class="icon-bar"></span>
+        </a>
+
+        <!-- Temporary logo placeholder -->
+        <a class="brand" href="#"><img src="/webui/static/img/finalasterixlogo.png"></a>
+
+        <div class="nav-collapse collapse">
+          <ul class="nav">
+            <li><a href="http://code.google.com/p/asterixdb/" target="_blank">
+                    Open source<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+            <li><a href="http://code.google.com/p/asterixdb/issues/list" target="_blank">
+                    File issues<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+            <li><a href="https://groups.google.com/forum/?fromgroups#!forum/asterixdb-users" target="_blank">
+                    Contact<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+          </ul>
+        </div><!--/.nav-collapse -->
+      </div>
+    </div>
+  </div>
+
+  <div class="content">
+    <div class="container">
+      <div class="row-fluid">
+       <div class="span12">
+       %s
+       </div>
+      </div>
+    </div>
+  </div>
+  <div class="footer">
+    <section class="line"><hr></section>
+    <section class="content">
+      <section class="left">
+      </section>
+      <section class="right">
+        &copy; Copyright 2013 University of California, Irvine
+      </section>
+    </section>
+  </div>
+</body>
+</html>
diff --git a/asterix-app/src/main/resources/feed/smoothie.js b/asterix-app/src/main/resources/feed/smoothie.js
new file mode 100644
index 0000000..4e46fa7
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/smoothie.js
@@ -0,0 +1,660 @@
+// MIT License:
+//
+// Copyright (c) 2010-2013, Joe Walnes
+//
+// Permission is hereby granted, free of charge, to any person obtaining a copy
+// of this software and associated documentation files (the "Software"), to deal
+// in the Software without restriction, including without limitation the rights
+// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+// copies of the Software, and to permit persons to whom the Software is
+// furnished to do so, subject to the following conditions:
+//
+// The above copyright notice and this permission notice shall be included in
+// all copies or substantial portions of the Software.
+//
+// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+// THE SOFTWARE.
+
+/**
+ * Smoothie Charts - http://smoothiecharts.org/
+ * (c) 2010-2013, Joe Walnes
+ *     2013, Drew Noakes
+ *
+ * v1.0: Main charting library, by Joe Walnes
+ * v1.1: Auto scaling of axis, by Neil Dunn
+ * v1.2: fps (frames per second) option, by Mathias Petterson
+ * v1.3: Fix for divide by zero, by Paul Nikitochkin
+ * v1.4: Set minimum, top-scale padding, remove timeseries, add optional timer to reset bounds, by Kelley Reynolds
+ * v1.5: Set default frames per second to 50... smoother.
+ *       .start(), .stop() methods for conserving CPU, by Dmitry Vyal
+ *       options.interpolation = 'bezier' or 'line', by Dmitry Vyal
+ *       options.maxValue to fix scale, by Dmitry Vyal
+ * v1.6: minValue/maxValue will always get converted to floats, by Przemek Matylla
+ * v1.7: options.grid.fillStyle may be a transparent color, by Dmitry A. Shashkin
+ *       Smooth rescaling, by Kostas Michalopoulos
+ * v1.8: Set max length to customize number of live points in the dataset with options.maxDataSetLength, by Krishna Narni
+ * v1.9: Display timestamps along the bottom, by Nick and Stev-io
+ *       (https://groups.google.com/forum/?fromgroups#!topic/smoothie-charts/-Ywse8FCpKI%5B1-25%5D)
+ *       Refactored by Krishna Narni, to support timestamp formatting function
+ * v1.10: Switch to requestAnimationFrame, removed the now obsoleted options.fps, by Gergely Imreh
+ * v1.11: options.grid.sharpLines option added, by @drewnoakes
+ *        Addressed warning seen in Firefox when seriesOption.fillStyle undefined, by @drewnoakes
+ * v1.12: Support for horizontalLines added, by @drewnoakes
+ *        Support for yRangeFunction callback added, by @drewnoakes
+ * v1.13: Fixed typo (#32), by @alnikitich
+ * v1.14: Timer cleared when last TimeSeries removed (#23), by @davidgaleano
+ *        Fixed diagonal line on chart at start/end of data stream, by @drewnoakes
+ * v1.15: Support for npm package (#18), by @dominictarr
+ *        Fixed broken removeTimeSeries function (#24) by @davidgaleano
+ *        Minor performance and tidying, by @drewnoakes
+ * v1.16: Bug fix introduced in v1.14 relating to timer creation/clearance (#23), by @drewnoakes
+ *        TimeSeries.append now deals with out-of-order timestamps, and can merge duplicates, by @zacwitte (#12)
+ *        Documentation and some local variable renaming for clarity, by @drewnoakes
+ * v1.17: Allow control over font size (#10), by @drewnoakes
+ *        Timestamp text won't overlap, by @drewnoakes
+ * v1.18: Allow control of max/min label precision, by @drewnoakes
+ *        Added 'borderVisible' chart option, by @drewnoakes
+ *        Allow drawing series with fill but no stroke (line), by @drewnoakes
+ */
+
+;(function(exports) {
+
+  var Util = {
+    extend: function() {
+      arguments[0] = arguments[0] || {};
+      for (var i = 1; i < arguments.length; i++)
+      {
+        for (var key in arguments[i])
+        {
+          if (arguments[i].hasOwnProperty(key))
+          {
+            if (typeof(arguments[i][key]) === 'object') {
+              if (arguments[i][key] instanceof Array) {
+                arguments[0][key] = arguments[i][key];
+              } else {
+                arguments[0][key] = Util.extend(arguments[0][key], arguments[i][key]);
+              }
+            } else {
+              arguments[0][key] = arguments[i][key];
+            }
+          }
+        }
+      }
+      return arguments[0];
+    }
+  };
+
+  /**
+   * Initialises a new <code>TimeSeries</code> with optional data options.
+   *
+   * Options are of the form (defaults shown):
+   *
+   * <pre>
+   * {
+   *   resetBounds: true,        // enables/disables automatic scaling of the y-axis
+   *   resetBoundsInterval: 3000 // the period between scaling calculations, in millis
+   * }
+   * </pre>
+   *
+   * Presentation options for TimeSeries are specified as an argument to <code>SmoothieChart.addTimeSeries</code>.
+   *
+   * @constructor
+   */
+  function TimeSeries(options) {
+    this.options = Util.extend({}, TimeSeries.defaultOptions, options);
+    this.data = [];
+    this.maxValue = Number.NaN; // The maximum value ever seen in this TimeSeries.
+    this.minValue = Number.NaN; // The minimum value ever seen in this TimeSeries.
+  }
+
+  TimeSeries.defaultOptions = {
+    resetBoundsInterval: 3000,
+    resetBounds: true
+  };
+
+  /**
+   * Recalculate the min/max values for this <code>TimeSeries</code> object.
+   *
+   * This causes the graph to scale itself in the y-axis.
+   */
+  TimeSeries.prototype.resetBounds = function() {
+    if (this.data.length) {
+      // Walk through all data points, finding the min/max value
+      this.maxValue = this.data[0][1];
+      this.minValue = this.data[0][1];
+      for (var i = 1; i < this.data.length; i++) {
+        var value = this.data[i][1];
+        if (value > this.maxValue) {
+          this.maxValue = value;
+        }
+        if (value < this.minValue) {
+          this.minValue = value;
+        }
+      }
+    } else {
+      // No data exists, so set min/max to NaN
+      this.maxValue = Number.NaN;
+      this.minValue = Number.NaN;
+    }
+  };
+
+  /**
+   * Adds a new data point to the <code>TimeSeries</code>, preserving chronological order.
+   *
+   * @param timestamp the position, in time, of this data point
+   * @param value the value of this data point
+   * @param sumRepeatedTimeStampValues if <code>timestamp</code> has an exact match in the series, this flag controls
+   * whether it is replaced, or the values summed (defaults to false.)
+   */
+  TimeSeries.prototype.append = function(timestamp, value, sumRepeatedTimeStampValues) {
+    // Rewind until we hit an older timestamp
+    var i = this.data.length - 1;
+    while (i > 0 && this.data[i][0] > timestamp) {
+      i--;
+    }
+
+    if (this.data.length > 0 && this.data[i][0] === timestamp) {
+      // Update existing values in the array
+      if (sumRepeatedTimeStampValues) {
+        // Sum this value into the existing 'bucket'
+        this.data[i][1] += value;
+        value = this.data[i][1];
+      } else {
+        // Replace the previous value
+        this.data[i][1] = value;
+      }
+    } else if (i < this.data.length - 1) {
+      // Splice into the correct position to keep timestamps in order
+      this.data.splice(i + 1, 0, [timestamp, value]);
+    } else {
+      // Add to the end of the array
+      this.data.push([timestamp, value]);
+    }
+
+    this.maxValue = isNaN(this.maxValue) ? value : Math.max(this.maxValue, value);
+    this.minValue = isNaN(this.minValue) ? value : Math.min(this.minValue, value);
+  };
+
+  TimeSeries.prototype.dropOldData = function(oldestValidTime, maxDataSetLength) {
+    // We must always keep one expired data point as we need this to draw the
+    // line that comes into the chart from the left, but any points prior to that can be removed.
+    var removeCount = 0;
+    while (this.data.length - removeCount >= maxDataSetLength && this.data[removeCount + 1][0] < oldestValidTime) {
+      removeCount++;
+    }
+    if (removeCount !== 0) {
+      this.data.splice(0, removeCount);
+    }
+  };
+
+  /**
+   * Initialises a new <code>SmoothieChart</code>.
+   *
+   * Options are optional, and should be of the form below. Just specify the values you
+   * need and the rest will be given sensible defaults as shown:
+   *
+   * <pre>
+   * {
+   *   minValue: undefined,        // specify to clamp the lower y-axis to a given value
+   *   maxValue: undefined,        // specify to clamp the upper y-axis to a given value
+   *   maxValueScale: 1,           // allows proportional padding to be added above the chart. for 10% padding, specify 1.1.
+   *   yRangeFunction: undefined,  // function({min: , max: }) { return {min: , max: }; }
+   *   scaleSmoothing: 0.125,      // controls the rate at which y-value zoom animation occurs
+   *   millisPerPixel: 20,         // sets the speed at which the chart pans by
+   *   maxDataSetLength: 2,
+   *   interpolation: 'bezier'     // or 'linear'
+   *   timestampFormatter: null,   // Optional function to format time stamps for bottom of chart. You may use SmoothieChart.timeFormatter, or your own: function(date) { return ''; }
+   *   horizontalLines: [],        // [ { value: 0, color: '#ffffff', lineWidth: 1 } ],
+   *   grid:
+   *   {
+   *     fillStyle: '#000000',     // the background colour of the chart
+   *     lineWidth: 1,             // the pixel width of grid lines
+   *     strokeStyle: '#777777',   // colour of grid lines
+   *     millisPerLine: 1000,      // distance between vertical grid lines
+   *     sharpLines: false,        // controls whether grid lines are 1px sharp, or softened
+   *     verticalSections: 2,      // number of vertical sections marked out by horizontal grid lines
+   *     borderVisible: true       // whether the grid lines trace the border of the chart or not
+   *   },
+   *   labels
+   *   {
+   *     disabled: false,          // enables/disables labels showing the min/max values
+   *     fillStyle: '#ffffff',     // colour for text of labels,
+   *     fontSize: 15,
+   *     fontFamily: 'sans-serif',
+   *     precision: 2
+   *   },
+   * }
+   * </pre>
+   *
+   * @constructor
+   */
+  function SmoothieChart(options) {
+    this.options = Util.extend({}, SmoothieChart.defaultChartOptions, options);
+    this.seriesSet = [];
+    this.currentValueRange = 1;
+    this.currentVisMinValue = 0;
+  }
+
+  SmoothieChart.defaultChartOptions = {
+    millisPerPixel: 20,
+    maxValueScale: 1,
+    interpolation: 'bezier',
+    scaleSmoothing: 0.125,
+    maxDataSetLength: 2,
+    grid: {
+      fillStyle: '#000000',
+      strokeStyle: '#777777',
+      lineWidth: 1,
+      sharpLines: false,
+      millisPerLine: 1000,
+      verticalSections: 2,
+      borderVisible: true
+    },
+    labels: {
+      fillStyle: '#ffffff',
+      disabled: false,
+      fontSize: 10,
+      fontFamily: 'monospace',
+      precision: 2
+    },
+    horizontalLines: []
+  };
+
+  // Based on http://inspirit.github.com/jsfeat/js/compatibility.js
+  SmoothieChart.AnimateCompatibility = (function() {
+    // TODO this global variable will cause bugs if more than one chart is used and the browser does not support *requestAnimationFrame natively
+    var lastTime = 0,
+        requestAnimationFrame = function(callback, element) {
+          var requestAnimationFrame =
+            window.requestAnimationFrame        ||
+            window.webkitRequestAnimationFrame  ||
+            window.mozRequestAnimationFrame     ||
+            window.oRequestAnimationFrame       ||
+            window.msRequestAnimationFrame      ||
+            function(callback) {
+              var currTime = new Date().getTime(),
+                  timeToCall = Math.max(0, 16 - (currTime - lastTime)),
+                  id = window.setTimeout(function() {
+                    callback(currTime + timeToCall);
+                  }, timeToCall);
+              lastTime = currTime + timeToCall;
+              return id;
+            };
+          return requestAnimationFrame.call(window, callback, element);
+        },
+        cancelAnimationFrame = function(id) {
+          var cancelAnimationFrame =
+            window.cancelAnimationFrame ||
+            function(id) {
+              clearTimeout(id);
+            };
+          return cancelAnimationFrame.call(window, id);
+        };
+
+    return {
+      requestAnimationFrame: requestAnimationFrame,
+      cancelAnimationFrame: cancelAnimationFrame
+    };
+  })();
+
+  SmoothieChart.defaultSeriesPresentationOptions = {
+    lineWidth: 1,
+    strokeStyle: '#ffffff'
+  };
+
+  /**
+   * Adds a <code>TimeSeries</code> to this chart, with optional presentation options.
+   *
+   * Presentation options should be of the form (defaults shown):
+   *
+   * <pre>
+   * {
+   *   lineWidth: 1,
+   *   strokeStyle: '#ffffff',
+   *   fillStyle: undefined
+   * }
+   * </pre>
+   */
+  SmoothieChart.prototype.addTimeSeries = function(timeSeries, options) {
+    this.seriesSet.push({timeSeries: timeSeries, options: Util.extend({}, SmoothieChart.defaultSeriesPresentationOptions, options)});
+    if (timeSeries.options.resetBounds && timeSeries.options.resetBoundsInterval > 0) {
+      timeSeries.resetBoundsTimerId = setInterval(
+        function() {
+          timeSeries.resetBounds();
+        },
+        timeSeries.options.resetBoundsInterval
+      );
+    }
+  };
+
+  /**
+   * Removes the specified <code>TimeSeries</code> from the chart.
+   */
+  SmoothieChart.prototype.removeTimeSeries = function(timeSeries) {
+    // Find the correct timeseries to remove, and remove it
+    var numSeries = this.seriesSet.length;
+    for (var i = 0; i < numSeries; i++) {
+      if (this.seriesSet[i].timeSeries === timeSeries) {
+        this.seriesSet.splice(i, 1);
+        break;
+      }
+    }
+    // If a timer was operating for that timeseries, remove it
+    if (timeSeries.resetBoundsTimerId) {
+      // Stop resetting the bounds, if we were
+      clearInterval(timeSeries.resetBoundsTimerId);
+    }
+  };
+
+  /**
+   * Instructs the <code>SmoothieChart</code> to start rendering to the provided canvas, with specified delay.
+   *
+   * @param canvas the target canvas element
+   * @param delayMillis an amount of time to wait before a data point is shown. This can prevent the end of the series
+   * from appearing on screen, with new values flashing into view, at the expense of some latency.
+   */
+  SmoothieChart.prototype.streamTo = function(canvas, delayMillis) {
+    this.canvas = canvas;
+    this.delay = delayMillis;
+    this.start();
+  };
+
+  /**
+   * Starts the animation of this chart.
+   */
+  SmoothieChart.prototype.start = function() {
+    if (this.frame) {
+      // We're already running, so just return
+      return;
+    }
+
+    // Renders a frame, and queues the next frame for later rendering
+    var animate = function() {
+      this.frame = SmoothieChart.AnimateCompatibility.requestAnimationFrame(function() {
+        this.render();
+        animate();
+      }.bind(this));
+    }.bind(this);
+
+    animate();
+  };
+
+  /**
+   * Stops the animation of this chart.
+   */
+  SmoothieChart.prototype.stop = function() {
+    if (this.frame) {
+      SmoothieChart.AnimateCompatibility.cancelAnimationFrame(this.frame);
+      delete this.frame;
+    }
+  };
+
+  SmoothieChart.prototype.updateValueRange = function() {
+    // Calculate the current scale of the chart, from all time series.
+    var chartOptions = this.options,
+        chartMaxValue = Number.NaN,
+        chartMinValue = Number.NaN;
+
+    for (var d = 0; d < this.seriesSet.length; d++) {
+      // TODO(ndunn): We could calculate / track these values as they stream in.
+      var timeSeries = this.seriesSet[d].timeSeries;
+      if (!isNaN(timeSeries.maxValue)) {
+        chartMaxValue = !isNaN(chartMaxValue) ? Math.max(chartMaxValue, timeSeries.maxValue) : timeSeries.maxValue;
+      }
+
+      if (!isNaN(timeSeries.minValue)) {
+        chartMinValue = !isNaN(chartMinValue) ? Math.min(chartMinValue, timeSeries.minValue) : timeSeries.minValue;
+      }
+    }
+
+    // Scale the chartMaxValue to add padding at the top if required
+    if (chartOptions.maxValue != null) {
+      chartMaxValue = chartOptions.maxValue;
+    } else {
+      chartMaxValue *= chartOptions.maxValueScale;
+    }
+
+    // Set the minimum if we've specified one
+    if (chartOptions.minValue != null) {
+      chartMinValue = chartOptions.minValue;
+    }
+
+    // If a custom range function is set, call it
+    if (this.options.yRangeFunction) {
+      var range = this.options.yRangeFunction({min: chartMinValue, max: chartMaxValue});
+      chartMinValue = range.min;
+      chartMaxValue = range.max;
+    }
+
+    if (!isNaN(chartMaxValue) && !isNaN(chartMinValue)) {
+      var targetValueRange = chartMaxValue - chartMinValue;
+      this.currentValueRange += chartOptions.scaleSmoothing * (targetValueRange - this.currentValueRange);
+      this.currentVisMinValue += chartOptions.scaleSmoothing * (chartMinValue - this.currentVisMinValue);
+    }
+
+    this.valueRange = { min: chartMinValue, max: chartMaxValue };
+  };
+
+  SmoothieChart.prototype.render = function(canvas, time) {
+    canvas = canvas || this.canvas;
+    time = time || new Date().getTime() - (this.delay || 0);
+
+    // TODO only render if the chart has moved at least 1px since the last rendered frame
+
+    // Round time down to pixel granularity, so motion appears smoother.
+    time -= time % this.options.millisPerPixel;
+
+    var context = canvas.getContext('2d'),
+        chartOptions = this.options,
+        dimensions = { top: 0, left: 0, width: canvas.clientWidth, height: canvas.clientHeight },
+        // Calculate the threshold time for the oldest data points.
+        oldestValidTime = time - (dimensions.width * chartOptions.millisPerPixel),
+        valueToYPixel = function(value) {
+          var offset = value - this.currentVisMinValue;
+          return this.currentValueRange === 0
+            ? dimensions.height
+            : dimensions.height - (Math.round((offset / this.currentValueRange) * dimensions.height));
+        }.bind(this),
+        timeToXPixel = function(t) {
+          return Math.round(dimensions.width - ((time - t) / chartOptions.millisPerPixel));
+        };
+
+    this.updateValueRange();
+
+    context.font = chartOptions.labels.fontSize + 'px ' + chartOptions.labels.fontFamily;
+
+    // Save the state of the canvas context, any transformations applied in this method
+    // will get removed from the stack at the end of this method when .restore() is called.
+    context.save();
+
+    // Move the origin.
+    context.translate(dimensions.left, dimensions.top);
+
+    // Create a clipped rectangle - anything we draw will be constrained to this rectangle.
+    // This prevents the occasional pixels from curves near the edges overrunning and creating
+    // screen cheese (that phrase should need no explanation).
+    context.beginPath();
+    context.rect(0, 0, dimensions.width, dimensions.height);
+    context.clip();
+
+    // Clear the working area.
+    context.save();
+    context.fillStyle = chartOptions.grid.fillStyle;
+    context.clearRect(0, 0, dimensions.width, dimensions.height);
+    context.fillRect(0, 0, dimensions.width, dimensions.height);
+    context.restore();
+
+    // Grid lines...
+    context.save();
+    context.lineWidth = chartOptions.grid.lineWidth;
+    context.strokeStyle = chartOptions.grid.strokeStyle;
+    // Vertical (time) dividers.
+    if (chartOptions.grid.millisPerLine > 0) {
+      var textUntilX = dimensions.width - context.measureText(minValueString).width + 4;
+      for (var t = time - (time % chartOptions.grid.millisPerLine);
+           t >= oldestValidTime;
+           t -= chartOptions.grid.millisPerLine) {
+        var gx = timeToXPixel(t);
+        if (chartOptions.grid.sharpLines) {
+          gx -= 0.5;
+        }
+        context.beginPath();
+        context.moveTo(gx, 0);
+        context.lineTo(gx, dimensions.height);
+        context.stroke();
+        context.closePath();
+
+        // Display timestamp at bottom of this line if requested, and it won't overlap
+        if (chartOptions.timestampFormatter && gx < textUntilX) {
+          // Formats the timestamp based on user specified formatting function
+          // SmoothieChart.timeFormatter function above is one such formatting option
+          var tx = new Date(t),
+            ts = chartOptions.timestampFormatter(tx),
+            tsWidth = context.measureText(ts).width;
+          textUntilX = gx - tsWidth - 2;
+          context.fillStyle = chartOptions.labels.fillStyle;
+          context.fillText(ts, gx - tsWidth, dimensions.height - 2);
+        }
+      }
+    }
+
+    // Horizontal (value) dividers.
+    for (var v = 1; v < chartOptions.grid.verticalSections; v++) {
+      var gy = Math.round(v * dimensions.height / chartOptions.grid.verticalSections);
+      if (chartOptions.grid.sharpLines) {
+        gy -= 0.5;
+      }
+      context.beginPath();
+      context.moveTo(0, gy);
+      context.lineTo(dimensions.width, gy);
+      context.stroke();
+      context.closePath();
+    }
+    // Bounding rectangle.
+    if (chartOptions.grid.borderVisible) {
+      context.beginPath();
+      context.strokeRect(0, 0, dimensions.width, dimensions.height);
+      context.closePath();
+    }
+    context.restore();
+
+    // Draw any horizontal lines...
+    if (chartOptions.horizontalLines && chartOptions.horizontalLines.length) {
+      for (var hl = 0; hl < chartOptions.horizontalLines.length; hl++) {
+        var line = chartOptions.horizontalLines[hl],
+            hly = Math.round(valueToYPixel(line.value)) - 0.5;
+        context.strokeStyle = line.color || '#ffffff';
+        context.lineWidth = line.lineWidth || 1;
+        context.beginPath();
+        context.moveTo(0, hly);
+        context.lineTo(dimensions.width, hly);
+        context.stroke();
+        context.closePath();
+      }
+    }
+
+    // For each data set...
+    for (var d = 0; d < this.seriesSet.length; d++) {
+      context.save();
+      var timeSeries = this.seriesSet[d].timeSeries,
+          dataSet = timeSeries.data,
+          seriesOptions = this.seriesSet[d].options;
+
+      // Delete old data that's moved off the left of the chart.
+      timeSeries.dropOldData(oldestValidTime, chartOptions.maxDataSetLength);
+
+      // Set style for this dataSet.
+      context.lineWidth = seriesOptions.lineWidth;
+      context.strokeStyle = seriesOptions.strokeStyle;
+      // Draw the line...
+      context.beginPath();
+      // Retain lastX, lastY for calculating the control points of bezier curves.
+      var firstX = 0, lastX = 0, lastY = 0;
+      for (var i = 0; i < dataSet.length && dataSet.length !== 1; i++) {
+        var x = timeToXPixel(dataSet[i][0]),
+            y = valueToYPixel(dataSet[i][1]);
+
+        if (i === 0) {
+          firstX = x;
+          context.moveTo(x, y);
+        } else {
+          switch (chartOptions.interpolation) {
+            case "linear":
+            case "line": {
+              context.lineTo(x,y);
+              break;
+            }
+            case "bezier":
+            default: {
+              // Great explanation of Bezier curves: http://en.wikipedia.org/wiki/Bezier_curve#Quadratic_curves
+              //
+              // Assuming A was the last point in the line plotted and B is the new point,
+              // we draw a curve with control points P and Q as below.
+              //
+              // A---P
+              //     |
+              //     |
+              //     |
+              //     Q---B
+              //
+              // Importantly, A and P are at the same y coordinate, as are B and Q. This is
+              // so adjacent curves appear to flow as one.
+              //
+              context.bezierCurveTo( // startPoint (A) is implicit from last iteration of loop
+                Math.round((lastX + x) / 2), lastY, // controlPoint1 (P)
+                Math.round((lastX + x)) / 2, y, // controlPoint2 (Q)
+                x, y); // endPoint (B)
+              break;
+            }
+          }
+        }
+
+        lastX = x; lastY = y;
+      }
+
+      if (dataSet.length > 1) {
+        if (seriesOptions.fillStyle) {
+          // Close up the fill region.
+          context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, lastY);
+          context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, dimensions.height + seriesOptions.lineWidth + 1);
+          context.lineTo(firstX, dimensions.height + seriesOptions.lineWidth);
+          context.fillStyle = seriesOptions.fillStyle;
+          context.fill();
+        }
+
+        if (seriesOptions.strokeStyle && seriesOptions.strokeStyle !== 'none') {
+          context.stroke();
+        }
+        context.closePath();
+      }
+      context.restore();
+    }
+
+    // Draw the axis values on the chart.
+    if (!chartOptions.labels.disabled && !isNaN(this.valueRange.min) && !isNaN(this.valueRange.max)) {
+      var maxValueString = parseFloat(this.valueRange.max).toFixed(chartOptions.labels.precision),
+          minValueString = parseFloat(this.valueRange.min).toFixed(chartOptions.labels.precision);
+      context.fillStyle = chartOptions.labels.fillStyle;
+      context.fillText(maxValueString, dimensions.width - context.measureText(maxValueString).width - 2, chartOptions.labels.fontSize);
+      context.fillText(minValueString, dimensions.width - context.measureText(minValueString).width - 2, dimensions.height - 2);
+    }
+
+    context.restore(); // See .save() above.
+  };
+
+  // Sample timestamp formatting function
+  SmoothieChart.timeFormatter = function(date) {
+    function pad2(number) { return (number < 10 ? '0' : '') + number }
+    return pad2(date.getHours()) + ':' + pad2(date.getMinutes()) + ':' + pad2(date.getSeconds());
+  };
+
+  exports.TimeSeries = TimeSeries;
+  exports.SmoothieChart = SmoothieChart;
+
+})(typeof exports === 'undefined' ?  this : exports);
+
diff --git a/asterix-app/src/main/resources/test.properties b/asterix-app/src/main/resources/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js b/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
index b4303a6..2cff1f8 100644
--- a/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
+++ b/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
@@ -1,7 +1 @@
-/*!
-	jQuery Autosize v1.16.7
-	(c) 2013 Jack Moore - jacklmoore.com
-	updated: 2013-03-20
-	license: http://www.opensource.org/licenses/mit-license.php
-*/
-(function(e){var t,o,n={className:"autosizejs",append:"",callback:!1},i="hidden",s="border-box",a="lineHeight",l='<textarea tabindex="-1" style="position:absolute; top:-999px; left:0; right:auto; bottom:auto; border:0; -moz-box-sizing:content-box; -webkit-box-sizing:content-box; box-sizing:content-box; word-wrap:break-word; height:0 !important; min-height:0 !important; overflow:hidden;"/>',r=["fontFamily","fontSize","fontWeight","fontStyle","letterSpacing","textTransform","wordSpacing","textIndent"],c="oninput",h="onpropertychange",p=e(l).data("autosize",!0)[0];p.style.lineHeight="99px","99px"===e(p).css(a)&&r.push(a),p.style.lineHeight="",e.fn.autosize=function(a){return a=e.extend({},n,a||{}),p.parentNode!==document.body&&(e(document.body).append(p),p.value="\n\n\n",p.scrollTop=9e4,t=p.scrollHeight===p.scrollTop+p.clientHeight),this.each(function(){function n(){o=b,p.className=a.className,e.each(r,function(e,t){p.style[t]=f.css(t)})}function l(){var e,s,l;if(o!==b&&n(),!d){d=!0,p.value=b.value+a.append,p.style.overflowY=b.style.overflowY,l=parseInt(b.style.height,10),p.style.width=Math.max(f.width(),0)+"px",t?e=p.scrollHeight:(p.scrollTop=0,p.scrollTop=9e4,e=p.scrollTop);var r=parseInt(f.css("maxHeight"),10);r=r&&r>0?r:9e4,e>r?(e=r,s="scroll"):u>e&&(e=u),e+=x,b.style.overflowY=s||i,l!==e&&(b.style.height=e+"px",w&&a.callback.call(b)),setTimeout(function(){d=!1},1)}}var u,d,g,b=this,f=e(b),x=0,w=e.isFunction(a.callback);f.data("autosize")||((f.css("box-sizing")===s||f.css("-moz-box-sizing")===s||f.css("-webkit-box-sizing")===s)&&(x=f.outerHeight()-f.height()),u=Math.max(parseInt(f.css("minHeight"),10)-x,f.height()),g="none"===f.css("resize")||"vertical"===f.css("resize")?"none":"horizontal",f.css({overflow:i,overflowY:i,wordWrap:"break-word",resize:g}).data("autosize",!0),h in b?c in b?b[c]=b.onkeyup=l:b[h]=l:b[c]=l,e(window).on("resize",function(){d=!1,l()}),f.on("autosize",function(){d=!1,l()}),l())})}})(window.jQuery||window.Zepto);
\ No newline at end of file
+(function(e){var t,o,n={className:"autosizejs",append:"",callback:!1},i="hidden",s="border-box",a="lineHeight",l='<textarea tabindex="-1" style="position:absolute; top:-999px; left:0; right:auto; bottom:auto; border:0; -moz-box-sizing:content-box; -webkit-box-sizing:content-box; box-sizing:content-box; word-wrap:break-word; height:0 !important; min-height:0 !important; overflow:hidden;"/>',r=["fontFamily","fontSize","fontWeight","fontStyle","letterSpacing","textTransform","wordSpacing","textIndent"],c="oninput",h="onpropertychange",p=e(l).data("autosize",!0)[0];p.style.lineHeight="99px","99px"===e(p).css(a)&&r.push(a),p.style.lineHeight="",e.fn.autosize=function(a){return a=e.extend({},n,a||{}),p.parentNode!==document.body&&(e(document.body).append(p),p.value="\n\n\n",p.scrollTop=9e4,t=p.scrollHeight===p.scrollTop+p.clientHeight),this.each(function(){function n(){o=b,p.className=a.className,e.each(r,function(e,t){p.style[t]=f.css(t)})}function l(){var e,s,l;if(o!==b&&n(),!d){d=!0,p.value=b.value+a.append,p.style.overflowY=b.style.overflowY,l=parseInt(b.style.height,10),p.style.width=Math.max(f.width(),0)+"px",t?e=p.scrollHeight:(p.scrollTop=0,p.scrollTop=9e4,e=p.scrollTop);var r=parseInt(f.css("maxHeight"),10);r=r&&r>0?r:9e4,e>r?(e=r,s="scroll"):u>e&&(e=u),e+=x,b.style.overflowY=s||i,l!==e&&(b.style.height=e+"px",w&&a.callback.call(b)),setTimeout(function(){d=!1},1)}}var u,d,g,b=this,f=e(b),x=0,w=e.isFunction(a.callback);f.data("autosize")||((f.css("box-sizing")===s||f.css("-moz-box-sizing")===s||f.css("-webkit-box-sizing")===s)&&(x=f.outerHeight()-f.height()),u=Math.max(parseInt(f.css("minHeight"),10)-x,f.height()),g="none"===f.css("resize")||"vertical"===f.css("resize")?"none":"horizontal",f.css({overflow:i,overflowY:i,wordWrap:"break-word",resize:g}).data("autosize",!0),h in b?c in b?b[c]=b.onkeyup=l:b[h]=l:b[c]=l,e(window).on("resize",function(){d=!1,l()}),f.on("autosize",function(){d=!1,l()}),l())})}})(window.jQuery||window.Zepto);
diff --git a/asterix-app/src/main/resources/webui/static/js/jquery.min.js b/asterix-app/src/main/resources/webui/static/js/jquery.min.js
index 006e953..0292c87 100644
--- a/asterix-app/src/main/resources/webui/static/js/jquery.min.js
+++ b/asterix-app/src/main/resources/webui/static/js/jquery.min.js
@@ -1,5 +1,3 @@
-/*! jQuery v1.9.1 | (c) 2005, 2012 jQuery Foundation, Inc. | jquery.org/license
-//@ sourceMappingURL=jquery.min.map
-*/(function(e,t){var n,r,i=typeof t,o=e.document,a=e.location,s=e.jQuery,u=e.$,l={},c=[],p="1.9.1",f=c.concat,d=c.push,h=c.slice,g=c.indexOf,m=l.toString,y=l.hasOwnProperty,v=p.trim,b=function(e,t){return new b.fn.init(e,t,r)},x=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,w=/\S+/g,T=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,k=/^[\],:{}\s]*$/,E=/(?:^|:|,)(?:\s*\[)+/g,S=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,A=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,j=/^-ms-/,D=/-([\da-z])/gi,L=function(e,t){return t.toUpperCase()},H=function(e){(o.addEventListener||"load"===e.type||"complete"===o.readyState)&&(q(),b.ready())},q=function(){o.addEventListener?(o.removeEventListener("DOMContentLoaded",H,!1),e.removeEventListener("load",H,!1)):(o.detachEvent("onreadystatechange",H),e.detachEvent("onload",H))};b.fn=b.prototype={jquery:p,constructor:b,init:function(e,n,r){var i,a;if(!e)return this;if("string"==typeof e){if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof b?n[0]:n,b.merge(this,b.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:o,!0)),C.test(i[1])&&b.isPlainObject(n))for(i in n)b.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(a=o.getElementById(i[2]),a&&a.parentNode){if(a.id!==i[2])return r.find(e);this.length=1,this[0]=a}return this.context=o,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):b.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),b.makeArray(e,this))},selector:"",length:0,size:function(){return this.length},toArray:function(){return h.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=b.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:function(e,t){return b.each(this,e,t)},ready:function(e){return b.ready.promise().done(e),this},slice:function(){return this.pushStack(h.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(b.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:d,sort:[].sort,splice:[].splice},b.fn.init.prototype=b.fn,b.extend=b.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},u=1,l=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},u=2),"object"==typeof s||b.isFunction(s)||(s={}),l===u&&(s=this,--u);l>u;u++)if(null!=(o=arguments[u]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(b.isPlainObject(r)||(n=b.isArray(r)))?(n?(n=!1,a=e&&b.isArray(e)?e:[]):a=e&&b.isPlainObject(e)?e:{},s[i]=b.extend(c,a,r)):r!==t&&(s[i]=r));return s},b.extend({noConflict:function(t){return e.$===b&&(e.$=u),t&&e.jQuery===b&&(e.jQuery=s),b},isReady:!1,readyWait:1,holdReady:function(e){e?b.readyWait++:b.ready(!0)},ready:function(e){if(e===!0?!--b.readyWait:!b.isReady){if(!o.body)return setTimeout(b.ready);b.isReady=!0,e!==!0&&--b.readyWait>0||(n.resolveWith(o,[b]),b.fn.trigger&&b(o).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===b.type(e)},isArray:Array.isArray||function(e){return"array"===b.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?l[m.call(e)]||"object":typeof e},isPlainObject:function(e){if(!e||"object"!==b.type(e)||e.nodeType||b.isWindow(e))return!1;try{if(e.constructor&&!y.call(e,"constructor")&&!y.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(n){return!1}var r;for(r in e);return r===t||y.call(e,r)},isEmptyObject:function(e){var t;for(t in e)return!1;return!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||o;var r=C.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=b.buildFragment([e],t,i),i&&b(i).remove(),b.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=b.trim(n),n&&k.test(n.replace(S,"@").replace(A,"]").replace(E,"")))?Function("return "+n)():(b.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||b.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&b.trim(t)&&(e.execScript||function(t){e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(j,"ms-").replace(D,L)},nodeName:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:v&&!v.call("\ufeff\u00a0")?function(e){return null==e?"":v.call(e)}:function(e){return null==e?"":(e+"").replace(T,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?b.merge(n,"string"==typeof e?[e]:e):d.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(g)return g.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length=i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r&&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return f.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),b.isFunction(e)?(r=h.call(arguments,2),i=function(){return e.apply(n||this,r.concat(h.call(arguments)))},i.guid=e.guid=e.guid||b.guid++,i):t},access:function(e,n,r,i,o,a,s){var u=0,l=e.length,c=null==r;if("object"===b.type(r)){o=!0;for(u in r)b.access(e,n,u,r[u],!0,a,s)}else if(i!==t&&(o=!0,b.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(b(e),n)})),n))for(;l>u;u++)n(e[u],r,s?i:i.call(e[u],u,n(e[u],r)));return o?e:c?n.call(e):l?n(e[0],r):a},now:function(){return(new Date).getTime()}}),b.ready.promise=function(t){if(!n)if(n=b.Deferred(),"complete"===o.readyState)setTimeout(b.ready);else if(o.addEventListener)o.addEventListener("DOMContentLoaded",H,!1),e.addEventListener("load",H,!1);else{o.attachEvent("onreadystatechange",H),e.attachEvent("onload",H);var r=!1;try{r=null==e.frameElement&&o.documentElement}catch(i){}r&&r.doScroll&&function a(){if(!b.isReady){try{r.doScroll("left")}catch(e){return setTimeout(a,50)}q(),b.ready()}}()}return n.promise(t)},b.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){l["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=b.type(e);return b.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=b(o);var _={};function F(e){var t=_[e]={};return b.each(e.match(w)||[],function(e,n){t[n]=!0}),t}b.Callbacks=function(e){e="string"==typeof e?_[e]||F(e):b.extend({},e);var n,r,i,o,a,s,u=[],l=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=u.length,n=!0;u&&o>a;a++)if(u[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,u&&(l?l.length&&c(l.shift()):r?u=[]:p.disable())},p={add:function(){if(u){var t=u.length;(function i(t){b.each(t,function(t,n){var r=b.type(n);"function"===r?e.unique&&p.has(n)||u.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=u.length:r&&(s=t,c(r))}return this},remove:function(){return u&&b.each(arguments,function(e,t){var r;while((r=b.inArray(t,u,r))>-1)u.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?b.inArray(e,u)>-1:!(!u||!u.length)},empty:function(){return u=[],this},disable:function(){return u=l=r=t,this},disabled:function(){return!u},lock:function(){return l=t,r||p.disable(),this},locked:function(){return!l},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!u||i&&!l||(n?l.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),this},fired:function(){return!!i}};return p},b.extend({Deferred:function(e){var t=[["resolve","done",b.Callbacks("once memory"),"resolved"],["reject","fail",b.Callbacks("once memory"),"rejected"],["notify","progress",b.Callbacks("memory")]],n="pending",r={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return b.Deferred(function(n){b.each(t,function(t,o){var a=o[0],s=b.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&b.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?b.extend(e,r):r}},i={};return r.pipe=r.then,b.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&&e.call(i,i),i},when:function(e){var t=0,n=h.call(arguments),r=n.length,i=1!==r||e&&b.isFunction(e.promise)?r:0,o=1===i?e:b.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?h.call(arguments):r,n===s?o.notifyWith(t,n):--i||o.resolveWith(t,n)}},s,u,l;if(r>1)for(s=Array(r),u=Array(r),l=Array(r);r>t;t++)n[t]&&b.isFunction(n[t].promise)?n[t].promise().done(a(t,l,n)).fail(o.reject).progress(a(t,u,s)):--i;return i||o.resolveWith(l,n),o.promise()}}),b.support=function(){var t,n,r,a,s,u,l,c,p,f,d=o.createElement("div");if(d.setAttribute("className","t"),d.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*"),r=d.getElementsByTagName("a")[0],!n||!r||!n.length)return{};s=o.createElement("select"),l=s.appendChild(o.createElement("option")),a=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t={getSetAttribute:"t"!==d.className,leadingWhitespace:3===d.firstChild.nodeType,tbody:!d.getElementsByTagName("tbody").length,htmlSerialize:!!d.getElementsByTagName("link").length,style:/top/.test(r.getAttribute("style")),hrefNormalized:"/a"===r.getAttribute("href"),opacity:/^0.5/.test(r.style.opacity),cssFloat:!!r.style.cssFloat,checkOn:!!a.value,optSelected:l.selected,enctype:!!o.createElement("form").enctype,html5Clone:"<:nav></:nav>"!==o.createElement("nav").cloneNode(!0).outerHTML,boxModel:"CSS1Compat"===o.compatMode,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},a.checked=!0,t.noCloneChecked=a.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!l.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}a=o.createElement("input"),a.setAttribute("value",""),t.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),t.radioValue="t"===a.value,a.setAttribute("checked","t"),a.setAttribute("name","t"),u=o.createDocumentFragment(),u.appendChild(a),t.appendChecked=a.checked,t.checkClone=u.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bubbles"]=c in e||d.attributes[c].expando===!1;return d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip,b(function(){var n,r,a,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",u=o.getElementsByTagName("body")[0];u&&(n=o.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",u.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",a=d.getElementsByTagName("td"),a[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===a[0].offsetHeight,a[0].style.display="",a[1].style.display="none",t.reliableHiddenOffsets=p&&0===a[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",t.boxSizing=4===d.offsetWidth,t.doesNotIncludeMarginInBodyOffset=1!==u.offsetTop,e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(o.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapBlocks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(u.style.zoom=1)),u.removeChild(n),n=d=a=r=null)}),n=s=u=l=r=a=null,t}();var O=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,B=/([A-Z])/g;function P(e,n,r,i){if(b.acceptData(e)){var o,a,s=b.expando,u="string"==typeof n,l=e.nodeType,p=l?b.cache:e,f=l?e[s]:e[s]&&s;if(f&&p[f]&&(i||p[f].data)||!u||r!==t)return f||(l?e[s]=f=c.pop()||b.guid++:f=s),p[f]||(p[f]={},l||(p[f].toJSON=b.noop)),("object"==typeof n||"function"==typeof n)&&(i?p[f]=b.extend(p[f],n):p[f].data=b.extend(p[f].data,n)),o=p[f],i||(o.data||(o.data={}),o=o.data),r!==t&&(o[b.camelCase(n)]=r),u?(a=o[n],null==a&&(a=o[b.camelCase(n)])):a=o,a}}function R(e,t,n){if(b.acceptData(e)){var r,i,o,a=e.nodeType,s=a?b.cache:e,u=a?e[b.expando]:b.expando;if(s[u]){if(t&&(o=n?s[u]:s[u].data)){b.isArray(t)?t=t.concat(b.map(t,b.camelCase)):t in o?t=[t]:(t=b.camelCase(t),t=t in o?[t]:t.split(" "));for(r=0,i=t.length;i>r;r++)delete o[t[r]];if(!(n?$:b.isEmptyObject)(o))return}(n||(delete s[u].data,$(s[u])))&&(a?b.cleanData([e],!0):b.support.deleteExpando||s!=s.window?delete s[u]:s[u]=null)}}}b.extend({cache:{},expando:"jQuery"+(p+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(e){return e=e.nodeType?b.cache[e[b.expando]]:e[b.expando],!!e&&!$(e)},data:function(e,t,n){return P(e,t,n)},removeData:function(e,t){return R(e,t)},_data:function(e,t,n){return P(e,t,n,!0)},_removeData:function(e,t){return R(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&b.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),b.fn.extend({data:function(e,n){var r,i,o=this[0],a=0,s=null;if(e===t){if(this.length&&(s=b.data(o),1===o.nodeType&&!b._data(o,"parsedAttrs"))){for(r=o.attributes;r.length>a;a++)i=r[a].name,i.indexOf("data-")||(i=b.camelCase(i.slice(5)),W(o,i,s[i]));b._data(o,"parsedAttrs",!0)}return s}return"object"==typeof e?this.each(function(){b.data(this,e)}):b.access(this,function(n){return n===t?o?W(o,e,b.data(o,e)):null:(this.each(function(){b.data(this,e,n)}),t)},null,n,arguments.length>1,null,!0)},removeData:function(e){return this.each(function(){b.removeData(this,e)})}});function W(e,n,r){if(r===t&&1===e.nodeType){var i="data-"+n.replace(B,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:O.test(r)?b.parseJSON(r):r}catch(o){}b.data(e,n,r)}else r=t}return r}function $(e){var t;for(t in e)if(("data"!==t||!b.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}b.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=b._data(e,n),r&&(!i||b.isArray(r)?i=b._data(e,n,b.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=b.queue(e,t),r=n.length,i=n.shift(),o=b._queueHooks(e,t),a=function(){b.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),o.cur=i,i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return b._data(e,n)||b._data(e,n,{empty:b.Callbacks("once memory").add(function(){b._removeData(e,t+"queue"),b._removeData(e,n)})})}}),b.fn.extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>arguments.length?b.queue(this[0],e):n===t?this:this.each(function(){var t=b.queue(this,e,n);b._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&b.dequeue(this,e)})},dequeue:function(e){return this.each(function(){b.dequeue(this,e)})},delay:function(e,t){return e=b.fx?b.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=b.Deferred(),a=this,s=this.length,u=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=b._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(u));return u(),o.promise(n)}});var I,z,X=/[\t\r\n]/g,U=/\r/g,V=/^(?:input|select|textarea|button|object)$/i,Y=/^(?:a|area)$/i,J=/^(?:checked|selected|autofocus|autoplay|async|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped)$/i,G=/^(?:checked|selected)$/i,Q=b.support.getSetAttribute,K=b.support.input;b.fn.extend({attr:function(e,t){return b.access(this,b.attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){b.removeAttr(this,e)})},prop:function(e,t){return b.access(this,b.prop,e,t,arguments.length>1)},removeProp:function(e){return e=b.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,u="string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).addClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=b.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,u=0===arguments.length||"string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).removeClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?b.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return b.isFunction(e)?this.each(function(n){b(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=b(this),u=t,l=e.match(w)||[];while(o=l[a++])u=r?u:!s.hasClass(o),s[u?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&b._data(this,"__className__",this.className),this.className=this.className||e===!1?"":b._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(X," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=b.isFunction(e),this.each(function(n){var o,a=b(this);1===this.nodeType&&(o=i?e.call(this,n,a.val()):e,null==o?o="":"number"==typeof o?o+="":b.isArray(o)&&(o=b.map(o,function(e){return null==e?"":e+""})),r=b.valHooks[this.type]||b.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=b.valHooks[o.type]||b.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(U,""):null==n?"":n)}}}),b.extend({valHooks:{option:{get:function(e){var t=e.attributes.value;return!t||t.specified?e.value:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,u=0>i?s:o?i:0;for(;s>u;u++)if(n=r[u],!(!n.selected&&u!==i||(b.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&b.nodeName(n.parentNode,"optgroup"))){if(t=b(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n=b.makeArray(t);return b(e).find("option").each(function(){this.selected=b.inArray(b(this).val(),n)>=0}),n.length||(e.selectedIndex=-1),n}}},attr:function(e,n,r){var o,a,s,u=e.nodeType;if(e&&3!==u&&8!==u&&2!==u)return typeof e.getAttribute===i?b.prop(e,n,r):(a=1!==u||!b.isXMLDoc(e),a&&(n=n.toLowerCase(),o=b.attrHooks[n]||(J.test(n)?z:I)),r===t?o&&a&&"get"in o&&null!==(s=o.get(e,n))?s:(typeof e.getAttribute!==i&&(s=e.getAttribute(n)),null==s?t:s):null!==r?o&&a&&"set"in o&&(s=o.set(e,r,n))!==t?s:(e.setAttribute(n,r+""),r):(b.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(w);if(o&&1===e.nodeType)while(n=o[i++])r=b.propFix[n]||n,J.test(n)?!Q&&G.test(n)?e[b.camelCase("default-"+n)]=e[r]=!1:e[r]=!1:b.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!b.support.radioValue&&"radio"===t&&b.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!b.isXMLDoc(e),a&&(n=b.propFix[n]||n,o=b.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var n=e.getAttributeNode("tabindex");return n&&n.specified?parseInt(n.value,10):V.test(e.nodeName)||Y.test(e.nodeName)&&e.href?0:t}}}}),z={get:function(e,n){var r=b.prop(e,n),i="boolean"==typeof r&&e.getAttribute(n),o="boolean"==typeof r?K&&Q?null!=i:G.test(n)?e[b.camelCase("default-"+n)]:!!i:e.getAttributeNode(n);return o&&o.value!==!1?n.toLowerCase():t},set:function(e,t,n){return t===!1?b.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&b.propFix[n]||n,n):e[b.camelCase("default-"+n)]=e[n]=!0,n}},K&&Q||(b.attrHooks.value={get:function(e,n){var r=e.getAttributeNode(n);return b.nodeName(e,"input")?e.defaultValue:r&&r.specified?r.value:t},set:function(e,n,r){return b.nodeName(e,"input")?(e.defaultValue=n,t):I&&I.set(e,n,r)}}),Q||(I=b.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&("id"===n||"name"===n||"coords"===n?""!==r.value:r.specified)?r.value:t},set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},b.attrHooks.contenteditable={get:I.get,set:function(e,t,n){I.set(e,""===t?!1:t,n)}},b.each(["width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}})})),b.support.hrefNormalized||(b.each(["href","src","width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{get:function(e){var r=e.getAttribute(n,2);return null==r?t:r}})}),b.each(["href","src"],function(e,t){b.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}})),b.support.style||(b.attrHooks.style={get:function(e){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),b.support.optSelected||(b.propHooks.selected=b.extend(b.propHooks.selected,{get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}})),b.support.enctype||(b.propFix.enctype="encoding"),b.support.checkOn||b.each(["radio","checkbox"],function(){b.valHooks[this]={get:function(e){return null===e.getAttribute("value")?"on":e.value}}}),b.each(["radio","checkbox"],function(){b.valHooks[this]=b.extend(b.valHooks[this],{set:function(e,n){return b.isArray(n)?e.checked=b.inArray(b(e).val(),n)>=0:t}})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}b.event={global:{},add:function(e,n,r,o,a){var s,u,l,c,p,f,d,h,g,m,y,v=b._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=b.guid++),(u=v.events)||(u=v.events={}),(f=v.handle)||(f=v.handle=function(e){return typeof b===i||e&&b.event.triggered===e.type?t:b.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(w)||[""],l=n.length;while(l--)s=rt.exec(n[l])||[],g=y=s[1],m=(s[2]||"").split(".").sort(),p=b.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=b.event.special[g]||{},d=b.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&b.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=u[g])||(h=u[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),b.event.global[g]=!0;e=null}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,p,f,d,h,g,m=b.hasData(e)&&b._data(e);if(m&&(c=m.events)){t=(t||"").match(w)||[""],l=t.length;while(l--)if(s=rt.exec(t[l])||[],d=g=s[1],h=(s[2]||"").split(".").sort(),d){p=b.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),u=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace)||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));u&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||b.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)b.event.remove(e,d+t[l],n,r,!0);b.isEmptyObject(c)&&(delete m.handle,b._removeData(e,"events"))}},trigger:function(n,r,i,a){var s,u,l,c,p,f,d,h=[i||o],g=y.call(n,"type")?n.type:n,m=y.call(n,"namespace")?n.namespace.split("."):[];if(l=f=i=i||o,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+b.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),u=0>g.indexOf(":")&&"on"+g,n=n[b.expando]?n:new b.Event(g,"object"==typeof n&&n),n.isTrigger=!0,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)"+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:b.makeArray(r,[n]),p=b.event.special[g]||{},a||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!a&&!p.noBubble&&!b.isWindow(i)){for(c=p.delegateType||g,nt.test(c+g)||(l=l.parentNode);l;l=l.parentNode)h.push(l),f=l;f===(i.ownerDocument||o)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((l=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(b._data(l,"events")||{})[n.type]&&b._data(l,"handle"),s&&s.apply(l,r),s=u&&l[u],s&&b.acceptData(l)&&s.apply&&s.apply(l,r)===!1&&n.preventDefault();if(n.type=g,!(a||n.isDefaultPrevented()||p._default&&p._default.apply(i.ownerDocument,r)!==!1||"click"===g&&b.nodeName(i,"a")||!b.acceptData(i)||!u||!i[g]||b.isWindow(i))){f=i[u],f&&(i[u]=null),b.event.triggered=g;try{i[g]()}catch(v){}b.event.triggered=t,f&&(i[u]=f)}return n.result}},dispatch:function(e){e=b.event.fix(e);var n,r,i,o,a,s=[],u=h.call(arguments),l=(b._data(this,"events")||{})[e.type]||[],c=b.event.special[e.type]||{};if(u[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=b.event.handlers.call(this,e,l),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropagationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((b.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,u),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],u=n.delegateCount,l=e.target;if(u&&l.nodeType&&(!e.button||"click"!==e.type))for(;l!=this;l=l.parentNode||this)if(1===l.nodeType&&(l.disabled!==!0||"click"!==e.type)){for(o=[],a=0;u>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?b(r,this).index(l)>=0:b.find(r,this,null,[l]).length),o[r]&&o.push(i);o.length&&s.push({elem:l,handlers:o})}return n.length>u&&s.push({elem:this,handlers:n.slice(u)}),s},fix:function(e){if(e[b.expando])return e;var t,n,r,i=e.type,a=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new b.Event(a),t=r.length;while(t--)n=r[t],e[n]=a[n];return e.target||(e.target=a.srcElement||o),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,a):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,a,s=n.button,u=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||o,a=i.documentElement,r=i.body,e.pageX=n.clientX+(a&&a.scrollLeft||r&&r.scrollLeft||0)-(a&&a.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(a&&a.scrollTop||r&&r.scrollTop||0)-(a&&a.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&u&&(e.relatedTarget=u===e.target?n.toElement:u),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},click:{trigger:function(){return b.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t}},focus:{trigger:function(){if(this!==o.activeElement&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===o.activeElement&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=b.extend(new b.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?b.event.trigger(i,null,t):b.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},b.removeEvent=o.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},b.Event=function(e,n){return this instanceof b.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&b.extend(this,n),this.timeStamp=e&&e.timeStamp||b.now(),this[b.expando]=!0,t):new b.Event(e,n)},b.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagation()}},b.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){b.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;
+(function(e,t){var n,r,i=typeof t,o=e.document,a=e.location,s=e.jQuery,u=e.$,l={},c=[],p="1.9.1",f=c.concat,d=c.push,h=c.slice,g=c.indexOf,m=l.toString,y=l.hasOwnProperty,v=p.trim,b=function(e,t){return new b.fn.init(e,t,r)},x=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,w=/\S+/g,T=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,k=/^[\],:{}\s]*$/,E=/(?:^|:|,)(?:\s*\[)+/g,S=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,A=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,j=/^-ms-/,D=/-([\da-z])/gi,L=function(e,t){return t.toUpperCase()},H=function(e){(o.addEventListener||"load"===e.type||"complete"===o.readyState)&&(q(),b.ready())},q=function(){o.addEventListener?(o.removeEventListener("DOMContentLoaded",H,!1),e.removeEventListener("load",H,!1)):(o.detachEvent("onreadystatechange",H),e.detachEvent("onload",H))};b.fn=b.prototype={jquery:p,constructor:b,init:function(e,n,r){var i,a;if(!e)return this;if("string"==typeof e){if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof b?n[0]:n,b.merge(this,b.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:o,!0)),C.test(i[1])&&b.isPlainObject(n))for(i in n)b.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(a=o.getElementById(i[2]),a&&a.parentNode){if(a.id!==i[2])return r.find(e);this.length=1,this[0]=a}return this.context=o,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):b.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),b.makeArray(e,this))},selector:"",length:0,size:function(){return this.length},toArray:function(){return h.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=b.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:function(e,t){return b.each(this,e,t)},ready:function(e){return b.ready.promise().done(e),this},slice:function(){return this.pushStack(h.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(b.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:d,sort:[].sort,splice:[].splice},b.fn.init.prototype=b.fn,b.extend=b.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},u=1,l=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},u=2),"object"==typeof s||b.isFunction(s)||(s={}),l===u&&(s=this,--u);l>u;u++)if(null!=(o=arguments[u]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(b.isPlainObject(r)||(n=b.isArray(r)))?(n?(n=!1,a=e&&b.isArray(e)?e:[]):a=e&&b.isPlainObject(e)?e:{},s[i]=b.extend(c,a,r)):r!==t&&(s[i]=r));return s},b.extend({noConflict:function(t){return e.$===b&&(e.$=u),t&&e.jQuery===b&&(e.jQuery=s),b},isReady:!1,readyWait:1,holdReady:function(e){e?b.readyWait++:b.ready(!0)},ready:function(e){if(e===!0?!--b.readyWait:!b.isReady){if(!o.body)return setTimeout(b.ready);b.isReady=!0,e!==!0&&--b.readyWait>0||(n.resolveWith(o,[b]),b.fn.trigger&&b(o).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===b.type(e)},isArray:Array.isArray||function(e){return"array"===b.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?l[m.call(e)]||"object":typeof e},isPlainObject:function(e){if(!e||"object"!==b.type(e)||e.nodeType||b.isWindow(e))return!1;try{if(e.constructor&&!y.call(e,"constructor")&&!y.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(n){return!1}var r;for(r in e);return r===t||y.call(e,r)},isEmptyObject:function(e){var t;for(t in e)return!1;return!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||o;var r=C.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=b.buildFragment([e],t,i),i&&b(i).remove(),b.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=b.trim(n),n&&k.test(n.replace(S,"@").replace(A,"]").replace(E,"")))?Function("return "+n)():(b.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||b.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&b.trim(t)&&(e.execScript||function(t){e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(j,"ms-").replace(D,L)},nodeName:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:v&&!v.call("\ufeff\u00a0")?function(e){return null==e?"":v.call(e)}:function(e){return null==e?"":(e+"").replace(T,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?b.merge(n,"string"==typeof e?[e]:e):d.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(g)return g.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length=i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r&&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return f.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),b.isFunction(e)?(r=h.call(arguments,2),i=function(){return e.apply(n||this,r.concat(h.call(arguments)))},i.guid=e.guid=e.guid||b.guid++,i):t},access:function(e,n,r,i,o,a,s){var u=0,l=e.length,c=null==r;if("object"===b.type(r)){o=!0;for(u in r)b.access(e,n,u,r[u],!0,a,s)}else if(i!==t&&(o=!0,b.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(b(e),n)})),n))for(;l>u;u++)n(e[u],r,s?i:i.call(e[u],u,n(e[u],r)));return o?e:c?n.call(e):l?n(e[0],r):a},now:function(){return(new Date).getTime()}}),b.ready.promise=function(t){if(!n)if(n=b.Deferred(),"complete"===o.readyState)setTimeout(b.ready);else if(o.addEventListener)o.addEventListener("DOMContentLoaded",H,!1),e.addEventListener("load",H,!1);else{o.attachEvent("onreadystatechange",H),e.attachEvent("onload",H);var r=!1;try{r=null==e.frameElement&&o.documentElement}catch(i){}r&&r.doScroll&&function a(){if(!b.isReady){try{r.doScroll("left")}catch(e){return setTimeout(a,50)}q(),b.ready()}}()}return n.promise(t)},b.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){l["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=b.type(e);return b.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=b(o);var _={};function F(e){var t=_[e]={};return b.each(e.match(w)||[],function(e,n){t[n]=!0}),t}b.Callbacks=function(e){e="string"==typeof e?_[e]||F(e):b.extend({},e);var n,r,i,o,a,s,u=[],l=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=u.length,n=!0;u&&o>a;a++)if(u[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,u&&(l?l.length&&c(l.shift()):r?u=[]:p.disable())},p={add:function(){if(u){var t=u.length;(function i(t){b.each(t,function(t,n){var r=b.type(n);"function"===r?e.unique&&p.has(n)||u.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=u.length:r&&(s=t,c(r))}return this},remove:function(){return u&&b.each(arguments,function(e,t){var r;while((r=b.inArray(t,u,r))>-1)u.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?b.inArray(e,u)>-1:!(!u||!u.length)},empty:function(){return u=[],this},disable:function(){return u=l=r=t,this},disabled:function(){return!u},lock:function(){return l=t,r||p.disable(),this},locked:function(){return!l},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!u||i&&!l||(n?l.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),this},fired:function(){return!!i}};return p},b.extend({Deferred:function(e){var t=[["resolve","done",b.Callbacks("once memory"),"resolved"],["reject","fail",b.Callbacks("once memory"),"rejected"],["notify","progress",b.Callbacks("memory")]],n="pending",r={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return b.Deferred(function(n){b.each(t,function(t,o){var a=o[0],s=b.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&b.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?b.extend(e,r):r}},i={};return r.pipe=r.then,b.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&&e.call(i,i),i},when:function(e){var t=0,n=h.call(arguments),r=n.length,i=1!==r||e&&b.isFunction(e.promise)?r:0,o=1===i?e:b.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?h.call(arguments):r,n===s?o.notifyWith(t,n):--i||o.resolveWith(t,n)}},s,u,l;if(r>1)for(s=Array(r),u=Array(r),l=Array(r);r>t;t++)n[t]&&b.isFunction(n[t].promise)?n[t].promise().done(a(t,l,n)).fail(o.reject).progress(a(t,u,s)):--i;return i||o.resolveWith(l,n),o.promise()}}),b.support=function(){var t,n,r,a,s,u,l,c,p,f,d=o.createElement("div");if(d.setAttribute("className","t"),d.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*"),r=d.getElementsByTagName("a")[0],!n||!r||!n.length)return{};s=o.createElement("select"),l=s.appendChild(o.createElement("option")),a=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t={getSetAttribute:"t"!==d.className,leadingWhitespace:3===d.firstChild.nodeType,tbody:!d.getElementsByTagName("tbody").length,htmlSerialize:!!d.getElementsByTagName("link").length,style:/top/.test(r.getAttribute("style")),hrefNormalized:"/a"===r.getAttribute("href"),opacity:/^0.5/.test(r.style.opacity),cssFloat:!!r.style.cssFloat,checkOn:!!a.value,optSelected:l.selected,enctype:!!o.createElement("form").enctype,html5Clone:"<:nav></:nav>"!==o.createElement("nav").cloneNode(!0).outerHTML,boxModel:"CSS1Compat"===o.compatMode,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},a.checked=!0,t.noCloneChecked=a.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!l.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}a=o.createElement("input"),a.setAttribute("value",""),t.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),t.radioValue="t"===a.value,a.setAttribute("checked","t"),a.setAttribute("name","t"),u=o.createDocumentFragment(),u.appendChild(a),t.appendChecked=a.checked,t.checkClone=u.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bubbles"]=c in e||d.attributes[c].expando===!1;return d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip,b(function(){var n,r,a,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",u=o.getElementsByTagName("body")[0];u&&(n=o.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",u.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",a=d.getElementsByTagName("td"),a[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===a[0].offsetHeight,a[0].style.display="",a[1].style.display="none",t.reliableHiddenOffsets=p&&0===a[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",t.boxSizing=4===d.offsetWidth,t.doesNotIncludeMarginInBodyOffset=1!==u.offsetTop,e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(o.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapBlocks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(u.style.zoom=1)),u.removeChild(n),n=d=a=r=null)}),n=s=u=l=r=a=null,t}();var O=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,B=/([A-Z])/g;function P(e,n,r,i){if(b.acceptData(e)){var o,a,s=b.expando,u="string"==typeof n,l=e.nodeType,p=l?b.cache:e,f=l?e[s]:e[s]&&s;if(f&&p[f]&&(i||p[f].data)||!u||r!==t)return f||(l?e[s]=f=c.pop()||b.guid++:f=s),p[f]||(p[f]={},l||(p[f].toJSON=b.noop)),("object"==typeof n||"function"==typeof n)&&(i?p[f]=b.extend(p[f],n):p[f].data=b.extend(p[f].data,n)),o=p[f],i||(o.data||(o.data={}),o=o.data),r!==t&&(o[b.camelCase(n)]=r),u?(a=o[n],null==a&&(a=o[b.camelCase(n)])):a=o,a}}function R(e,t,n){if(b.acceptData(e)){var r,i,o,a=e.nodeType,s=a?b.cache:e,u=a?e[b.expando]:b.expando;if(s[u]){if(t&&(o=n?s[u]:s[u].data)){b.isArray(t)?t=t.concat(b.map(t,b.camelCase)):t in o?t=[t]:(t=b.camelCase(t),t=t in o?[t]:t.split(" "));for(r=0,i=t.length;i>r;r++)delete o[t[r]];if(!(n?$:b.isEmptyObject)(o))return}(n||(delete s[u].data,$(s[u])))&&(a?b.cleanData([e],!0):b.support.deleteExpando||s!=s.window?delete s[u]:s[u]=null)}}}b.extend({cache:{},expando:"jQuery"+(p+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(e){return e=e.nodeType?b.cache[e[b.expando]]:e[b.expando],!!e&&!$(e)},data:function(e,t,n){return P(e,t,n)},removeData:function(e,t){return R(e,t)},_data:function(e,t,n){return P(e,t,n,!0)},_removeData:function(e,t){return R(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&b.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),b.fn.extend({data:function(e,n){var r,i,o=this[0],a=0,s=null;if(e===t){if(this.length&&(s=b.data(o),1===o.nodeType&&!b._data(o,"parsedAttrs"))){for(r=o.attributes;r.length>a;a++)i=r[a].name,i.indexOf("data-")||(i=b.camelCase(i.slice(5)),W(o,i,s[i]));b._data(o,"parsedAttrs",!0)}return s}return"object"==typeof e?this.each(function(){b.data(this,e)}):b.access(this,function(n){return n===t?o?W(o,e,b.data(o,e)):null:(this.each(function(){b.data(this,e,n)}),t)},null,n,arguments.length>1,null,!0)},removeData:function(e){return this.each(function(){b.removeData(this,e)})}});function W(e,n,r){if(r===t&&1===e.nodeType){var i="data-"+n.replace(B,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:O.test(r)?b.parseJSON(r):r}catch(o){}b.data(e,n,r)}else r=t}return r}function $(e){var t;for(t in e)if(("data"!==t||!b.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}b.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=b._data(e,n),r&&(!i||b.isArray(r)?i=b._data(e,n,b.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=b.queue(e,t),r=n.length,i=n.shift(),o=b._queueHooks(e,t),a=function(){b.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),o.cur=i,i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return b._data(e,n)||b._data(e,n,{empty:b.Callbacks("once memory").add(function(){b._removeData(e,t+"queue"),b._removeData(e,n)})})}}),b.fn.extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>arguments.length?b.queue(this[0],e):n===t?this:this.each(function(){var t=b.queue(this,e,n);b._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&b.dequeue(this,e)})},dequeue:function(e){return this.each(function(){b.dequeue(this,e)})},delay:function(e,t){return e=b.fx?b.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=b.Deferred(),a=this,s=this.length,u=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=b._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(u));return u(),o.promise(n)}});var I,z,X=/[\t\r\n]/g,U=/\r/g,V=/^(?:input|select|textarea|button|object)$/i,Y=/^(?:a|area)$/i,J=/^(?:checked|selected|autofocus|autoplay|async|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped)$/i,G=/^(?:checked|selected)$/i,Q=b.support.getSetAttribute,K=b.support.input;b.fn.extend({attr:function(e,t){return b.access(this,b.attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){b.removeAttr(this,e)})},prop:function(e,t){return b.access(this,b.prop,e,t,arguments.length>1)},removeProp:function(e){return e=b.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,u="string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).addClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=b.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,u=0===arguments.length||"string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).removeClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?b.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return b.isFunction(e)?this.each(function(n){b(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=b(this),u=t,l=e.match(w)||[];while(o=l[a++])u=r?u:!s.hasClass(o),s[u?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&b._data(this,"__className__",this.className),this.className=this.className||e===!1?"":b._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(X," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=b.isFunction(e),this.each(function(n){var o,a=b(this);1===this.nodeType&&(o=i?e.call(this,n,a.val()):e,null==o?o="":"number"==typeof o?o+="":b.isArray(o)&&(o=b.map(o,function(e){return null==e?"":e+""})),r=b.valHooks[this.type]||b.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=b.valHooks[o.type]||b.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(U,""):null==n?"":n)}}}),b.extend({valHooks:{option:{get:function(e){var t=e.attributes.value;return!t||t.specified?e.value:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,u=0>i?s:o?i:0;for(;s>u;u++)if(n=r[u],!(!n.selected&&u!==i||(b.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&b.nodeName(n.parentNode,"optgroup"))){if(t=b(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n=b.makeArray(t);return b(e).find("option").each(function(){this.selected=b.inArray(b(this).val(),n)>=0}),n.length||(e.selectedIndex=-1),n}}},attr:function(e,n,r){var o,a,s,u=e.nodeType;if(e&&3!==u&&8!==u&&2!==u)return typeof e.getAttribute===i?b.prop(e,n,r):(a=1!==u||!b.isXMLDoc(e),a&&(n=n.toLowerCase(),o=b.attrHooks[n]||(J.test(n)?z:I)),r===t?o&&a&&"get"in o&&null!==(s=o.get(e,n))?s:(typeof e.getAttribute!==i&&(s=e.getAttribute(n)),null==s?t:s):null!==r?o&&a&&"set"in o&&(s=o.set(e,r,n))!==t?s:(e.setAttribute(n,r+""),r):(b.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(w);if(o&&1===e.nodeType)while(n=o[i++])r=b.propFix[n]||n,J.test(n)?!Q&&G.test(n)?e[b.camelCase("default-"+n)]=e[r]=!1:e[r]=!1:b.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!b.support.radioValue&&"radio"===t&&b.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!b.isXMLDoc(e),a&&(n=b.propFix[n]||n,o=b.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var n=e.getAttributeNode("tabindex");return n&&n.specified?parseInt(n.value,10):V.test(e.nodeName)||Y.test(e.nodeName)&&e.href?0:t}}}}),z={get:function(e,n){var r=b.prop(e,n),i="boolean"==typeof r&&e.getAttribute(n),o="boolean"==typeof r?K&&Q?null!=i:G.test(n)?e[b.camelCase("default-"+n)]:!!i:e.getAttributeNode(n);return o&&o.value!==!1?n.toLowerCase():t},set:function(e,t,n){return t===!1?b.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&b.propFix[n]||n,n):e[b.camelCase("default-"+n)]=e[n]=!0,n}},K&&Q||(b.attrHooks.value={get:function(e,n){var r=e.getAttributeNode(n);return b.nodeName(e,"input")?e.defaultValue:r&&r.specified?r.value:t},set:function(e,n,r){return b.nodeName(e,"input")?(e.defaultValue=n,t):I&&I.set(e,n,r)}}),Q||(I=b.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&("id"===n||"name"===n||"coords"===n?""!==r.value:r.specified)?r.value:t},set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},b.attrHooks.contenteditable={get:I.get,set:function(e,t,n){I.set(e,""===t?!1:t,n)}},b.each(["width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}})})),b.support.hrefNormalized||(b.each(["href","src","width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{get:function(e){var r=e.getAttribute(n,2);return null==r?t:r}})}),b.each(["href","src"],function(e,t){b.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}})),b.support.style||(b.attrHooks.style={get:function(e){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),b.support.optSelected||(b.propHooks.selected=b.extend(b.propHooks.selected,{get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}})),b.support.enctype||(b.propFix.enctype="encoding"),b.support.checkOn||b.each(["radio","checkbox"],function(){b.valHooks[this]={get:function(e){return null===e.getAttribute("value")?"on":e.value}}}),b.each(["radio","checkbox"],function(){b.valHooks[this]=b.extend(b.valHooks[this],{set:function(e,n){return b.isArray(n)?e.checked=b.inArray(b(e).val(),n)>=0:t}})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}b.event={global:{},add:function(e,n,r,o,a){var s,u,l,c,p,f,d,h,g,m,y,v=b._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=b.guid++),(u=v.events)||(u=v.events={}),(f=v.handle)||(f=v.handle=function(e){return typeof b===i||e&&b.event.triggered===e.type?t:b.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(w)||[""],l=n.length;while(l--)s=rt.exec(n[l])||[],g=y=s[1],m=(s[2]||"").split(".").sort(),p=b.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=b.event.special[g]||{},d=b.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&b.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=u[g])||(h=u[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),b.event.global[g]=!0;e=null}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,p,f,d,h,g,m=b.hasData(e)&&b._data(e);if(m&&(c=m.events)){t=(t||"").match(w)||[""],l=t.length;while(l--)if(s=rt.exec(t[l])||[],d=g=s[1],h=(s[2]||"").split(".").sort(),d){p=b.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),u=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace)||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));u&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||b.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)b.event.remove(e,d+t[l],n,r,!0);b.isEmptyObject(c)&&(delete m.handle,b._removeData(e,"events"))}},trigger:function(n,r,i,a){var s,u,l,c,p,f,d,h=[i||o],g=y.call(n,"type")?n.type:n,m=y.call(n,"namespace")?n.namespace.split("."):[];if(l=f=i=i||o,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+b.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),u=0>g.indexOf(":")&&"on"+g,n=n[b.expando]?n:new b.Event(g,"object"==typeof n&&n),n.isTrigger=!0,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)"+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:b.makeArray(r,[n]),p=b.event.special[g]||{},a||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!a&&!p.noBubble&&!b.isWindow(i)){for(c=p.delegateType||g,nt.test(c+g)||(l=l.parentNode);l;l=l.parentNode)h.push(l),f=l;f===(i.ownerDocument||o)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((l=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(b._data(l,"events")||{})[n.type]&&b._data(l,"handle"),s&&s.apply(l,r),s=u&&l[u],s&&b.acceptData(l)&&s.apply&&s.apply(l,r)===!1&&n.preventDefault();if(n.type=g,!(a||n.isDefaultPrevented()||p._default&&p._default.apply(i.ownerDocument,r)!==!1||"click"===g&&b.nodeName(i,"a")||!b.acceptData(i)||!u||!i[g]||b.isWindow(i))){f=i[u],f&&(i[u]=null),b.event.triggered=g;try{i[g]()}catch(v){}b.event.triggered=t,f&&(i[u]=f)}return n.result}},dispatch:function(e){e=b.event.fix(e);var n,r,i,o,a,s=[],u=h.call(arguments),l=(b._data(this,"events")||{})[e.type]||[],c=b.event.special[e.type]||{};if(u[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=b.event.handlers.call(this,e,l),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropagationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((b.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,u),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],u=n.delegateCount,l=e.target;if(u&&l.nodeType&&(!e.button||"click"!==e.type))for(;l!=this;l=l.parentNode||this)if(1===l.nodeType&&(l.disabled!==!0||"click"!==e.type)){for(o=[],a=0;u>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?b(r,this).index(l)>=0:b.find(r,this,null,[l]).length),o[r]&&o.push(i);o.length&&s.push({elem:l,handlers:o})}return n.length>u&&s.push({elem:this,handlers:n.slice(u)}),s},fix:function(e){if(e[b.expando])return e;var t,n,r,i=e.type,a=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new b.Event(a),t=r.length;while(t--)n=r[t],e[n]=a[n];return e.target||(e.target=a.srcElement||o),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,a):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,a,s=n.button,u=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||o,a=i.documentElement,r=i.body,e.pageX=n.clientX+(a&&a.scrollLeft||r&&r.scrollLeft||0)-(a&&a.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(a&&a.scrollTop||r&&r.scrollTop||0)-(a&&a.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&u&&(e.relatedTarget=u===e.target?n.toElement:u),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},click:{trigger:function(){return b.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t}},focus:{trigger:function(){if(this!==o.activeElement&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===o.activeElement&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=b.extend(new b.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?b.event.trigger(i,null,t):b.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},b.removeEvent=o.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},b.Event=function(e,n){return this instanceof b.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&b.extend(this,n),this.timeStamp=e&&e.timeStamp||b.now(),this[b.expando]=!0,t):new b.Event(e,n)},b.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagation()}},b.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){b.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;
 return(!i||i!==r&&!b.contains(r,i))&&(e.type=o.origType,n=o.handler.apply(this,arguments),e.type=t),n}}}),b.support.submitBubbles||(b.event.special.submit={setup:function(){return b.nodeName(this,"form")?!1:(b.event.add(this,"click._submit keypress._submit",function(e){var n=e.target,r=b.nodeName(n,"input")||b.nodeName(n,"button")?n.form:t;r&&!b._data(r,"submitBubbles")&&(b.event.add(r,"submit._submit",function(e){e._submit_bubble=!0}),b._data(r,"submitBubbles",!0))}),t)},postDispatch:function(e){e._submit_bubble&&(delete e._submit_bubble,this.parentNode&&!e.isTrigger&&b.event.simulate("submit",this.parentNode,e,!0))},teardown:function(){return b.nodeName(this,"form")?!1:(b.event.remove(this,"._submit"),t)}}),b.support.changeBubbles||(b.event.special.change={setup:function(){return Z.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(b.event.add(this,"propertychange._change",function(e){"checked"===e.originalEvent.propertyName&&(this._just_changed=!0)}),b.event.add(this,"click._change",function(e){this._just_changed&&!e.isTrigger&&(this._just_changed=!1),b.event.simulate("change",this,e,!0)})),!1):(b.event.add(this,"beforeactivate._change",function(e){var t=e.target;Z.test(t.nodeName)&&!b._data(t,"changeBubbles")&&(b.event.add(t,"change._change",function(e){!this.parentNode||e.isSimulated||e.isTrigger||b.event.simulate("change",this.parentNode,e,!0)}),b._data(t,"changeBubbles",!0))}),t)},handle:function(e){var n=e.target;return this!==n||e.isSimulated||e.isTrigger||"radio"!==n.type&&"checkbox"!==n.type?e.handleObj.handler.apply(this,arguments):t},teardown:function(){return b.event.remove(this,"._change"),!Z.test(this.nodeName)}}),b.support.focusinBubbles||b.each({focus:"focusin",blur:"focusout"},function(e,t){var n=0,r=function(e){b.event.simulate(t,e.target,b.event.fix(e),!0)};b.event.special[t]={setup:function(){0===n++&&o.addEventListener(e,r,!0)},teardown:function(){0===--n&&o.removeEventListener(e,r,!0)}}}),b.fn.extend({on:function(e,n,r,i,o){var a,s;if("object"==typeof e){"string"!=typeof n&&(r=r||n,n=t);for(a in e)this.on(a,n,r,e[a],o);return this}if(null==r&&null==i?(i=n,r=n=t):null==i&&("string"==typeof n?(i=r,r=t):(i=r,r=n,n=t)),i===!1)i=ot;else if(!i)return this;return 1===o&&(s=i,i=function(e){return b().off(e),s.apply(this,arguments)},i.guid=s.guid||(s.guid=b.guid++)),this.each(function(){b.event.add(this,e,i,r,n)})},one:function(e,t,n,r){return this.on(e,t,n,r,1)},off:function(e,n,r){var i,o;if(e&&e.preventDefault&&e.handleObj)return i=e.handleObj,b(e.delegateTarget).off(i.namespace?i.origType+"."+i.namespace:i.origType,i.selector,i.handler),this;if("object"==typeof e){for(o in e)this.off(o,n,e[o]);return this}return(n===!1||"function"==typeof n)&&(r=n,n=t),r===!1&&(r=ot),this.each(function(){b.event.remove(this,e,r,n)})},bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},trigger:function(e,t){return this.each(function(){b.event.trigger(e,t,this)})},triggerHandler:function(e,n){var r=this[0];return r?b.event.trigger(e,n,r,!0):t}}),function(e,t){var n,r,i,o,a,s,u,l,c,p,f,d,h,g,m,y,v,x="sizzle"+-new Date,w=e.document,T={},N=0,C=0,k=it(),E=it(),S=it(),A=typeof t,j=1<<31,D=[],L=D.pop,H=D.push,q=D.slice,M=D.indexOf||function(e){var t=0,n=this.length;for(;n>t;t++)if(this[t]===e)return t;return-1},_="[\\x20\\t\\r\\n\\f]",F="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=F.replace("w","w#"),B="([*^$|!~]?=)",P="\\["+_+"*("+F+")"+_+"*(?:"+B+_+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+O+")|)|)"+_+"*\\]",R=":("+F+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+P.replace(3,8)+")*)|.*)\\)|)",W=RegExp("^"+_+"+|((?:^|[^\\\\])(?:\\\\.)*)"+_+"+$","g"),$=RegExp("^"+_+"*,"+_+"*"),I=RegExp("^"+_+"*([\\x20\\t\\r\\n\\f>+~])"+_+"*"),z=RegExp(R),X=RegExp("^"+O+"$"),U={ID:RegExp("^#("+F+")"),CLASS:RegExp("^\\.("+F+")"),NAME:RegExp("^\\[name=['\"]?("+F+")['\"]?\\]"),TAG:RegExp("^("+F.replace("w","w*")+")"),ATTR:RegExp("^"+P),PSEUDO:RegExp("^"+R),CHILD:RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+_+"*(even|odd|(([+-]|)(\\d*)n|)"+_+"*(?:([+-]|)"+_+"*(\\d+)|))"+_+"*\\)|)","i"),needsContext:RegExp("^"+_+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+_+"*((?:-\\d)?\\d*)"+_+"*\\)|)(?=[^-]|$)","i")},V=/[\x20\t\r\n\f]*[+~]/,Y=/^[^{]+\{\s*\[native code/,J=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,G=/^(?:input|select|textarea|button)$/i,Q=/^h\d$/i,K=/'|\\/g,Z=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,et=/\\([\da-fA-F]{1,6}[\x20\t\r\n\f]?|.)/g,tt=function(e,t){var n="0x"+t-65536;return n!==n?t:0>n?String.fromCharCode(n+65536):String.fromCharCode(55296|n>>10,56320|1023&n)};try{q.call(w.documentElement.childNodes,0)[0].nodeType}catch(nt){q=function(e){var t,n=[];while(t=this[e++])n.push(t);return n}}function rt(e){return Y.test(e+"")}function it(){var e,t=[];return e=function(n,r){return t.push(n+=" ")>i.cacheLength&&delete e[t.shift()],e[n]=r}}function ot(e){return e[x]=!0,e}function at(e){var t=p.createElement("div");try{return e(t)}catch(n){return!1}finally{t=null}}function st(e,t,n,r){var i,o,a,s,u,l,f,g,m,v;if((t?t.ownerDocument||t:w)!==p&&c(t),t=t||p,n=n||[],!e||"string"!=typeof e)return n;if(1!==(s=t.nodeType)&&9!==s)return[];if(!d&&!r){if(i=J.exec(e))if(a=i[1]){if(9===s){if(o=t.getElementById(a),!o||!o.parentNode)return n;if(o.id===a)return n.push(o),n}else if(t.ownerDocument&&(o=t.ownerDocument.getElementById(a))&&y(t,o)&&o.id===a)return n.push(o),n}else{if(i[2])return H.apply(n,q.call(t.getElementsByTagName(e),0)),n;if((a=i[3])&&T.getByClassName&&t.getElementsByClassName)return H.apply(n,q.call(t.getElementsByClassName(a),0)),n}if(T.qsa&&!h.test(e)){if(f=!0,g=x,m=t,v=9===s&&e,1===s&&"object"!==t.nodeName.toLowerCase()){l=ft(e),(f=t.getAttribute("id"))?g=f.replace(K,"\\$&"):t.setAttribute("id",g),g="[id='"+g+"'] ",u=l.length;while(u--)l[u]=g+dt(l[u]);m=V.test(e)&&t.parentNode||t,v=l.join(",")}if(v)try{return H.apply(n,q.call(m.querySelectorAll(v),0)),n}catch(b){}finally{f||t.removeAttribute("id")}}}return wt(e.replace(W,"$1"),t,n,r)}a=st.isXML=function(e){var t=e&&(e.ownerDocument||e).documentElement;return t?"HTML"!==t.nodeName:!1},c=st.setDocument=function(e){var n=e?e.ownerDocument||e:w;return n!==p&&9===n.nodeType&&n.documentElement?(p=n,f=n.documentElement,d=a(n),T.tagNameNoComments=at(function(e){return e.appendChild(n.createComment("")),!e.getElementsByTagName("*").length}),T.attributes=at(function(e){e.innerHTML="<select></select>";var t=typeof e.lastChild.getAttribute("multiple");return"boolean"!==t&&"string"!==t}),T.getByClassName=at(function(e){return e.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",e.getElementsByClassName&&e.getElementsByClassName("e").length?(e.lastChild.className="e",2===e.getElementsByClassName("e").length):!1}),T.getByName=at(function(e){e.id=x+0,e.innerHTML="<a name='"+x+"'></a><div name='"+x+"'></div>",f.insertBefore(e,f.firstChild);var t=n.getElementsByName&&n.getElementsByName(x).length===2+n.getElementsByName(x+0).length;return T.getIdNotName=!n.getElementById(x),f.removeChild(e),t}),i.attrHandle=at(function(e){return e.innerHTML="<a href='#'></a>",e.firstChild&&typeof e.firstChild.getAttribute!==A&&"#"===e.firstChild.getAttribute("href")})?{}:{href:function(e){return e.getAttribute("href",2)},type:function(e){return e.getAttribute("type")}},T.getIdNotName?(i.find.ID=function(e,t){if(typeof t.getElementById!==A&&!d){var n=t.getElementById(e);return n&&n.parentNode?[n]:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){return e.getAttribute("id")===t}}):(i.find.ID=function(e,n){if(typeof n.getElementById!==A&&!d){var r=n.getElementById(e);return r?r.id===e||typeof r.getAttributeNode!==A&&r.getAttributeNode("id").value===e?[r]:t:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){var n=typeof e.getAttributeNode!==A&&e.getAttributeNode("id");return n&&n.value===t}}),i.find.TAG=T.tagNameNoComments?function(e,n){return typeof n.getElementsByTagName!==A?n.getElementsByTagName(e):t}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},i.find.NAME=T.getByName&&function(e,n){return typeof n.getElementsByName!==A?n.getElementsByName(name):t},i.find.CLASS=T.getByClassName&&function(e,n){return typeof n.getElementsByClassName===A||d?t:n.getElementsByClassName(e)},g=[],h=[":focus"],(T.qsa=rt(n.querySelectorAll))&&(at(function(e){e.innerHTML="<select><option selected=''></option></select>",e.querySelectorAll("[selected]").length||h.push("\\["+_+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),e.querySelectorAll(":checked").length||h.push(":checked")}),at(function(e){e.innerHTML="<input type='hidden' i=''/>",e.querySelectorAll("[i^='']").length&&h.push("[*^$]="+_+"*(?:\"\"|'')"),e.querySelectorAll(":enabled").length||h.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),h.push(",.*:")})),(T.matchesSelector=rt(m=f.matchesSelector||f.mozMatchesSelector||f.webkitMatchesSelector||f.oMatchesSelector||f.msMatchesSelector))&&at(function(e){T.disconnectedMatch=m.call(e,"div"),m.call(e,"[s!='']:x"),g.push("!=",R)}),h=RegExp(h.join("|")),g=RegExp(g.join("|")),y=rt(f.contains)||f.compareDocumentPosition?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},v=f.compareDocumentPosition?function(e,t){var r;return e===t?(u=!0,0):(r=t.compareDocumentPosition&&e.compareDocumentPosition&&e.compareDocumentPosition(t))?1&r||e.parentNode&&11===e.parentNode.nodeType?e===n||y(w,e)?-1:t===n||y(w,t)?1:0:4&r?-1:1:e.compareDocumentPosition?-1:1}:function(e,t){var r,i=0,o=e.parentNode,a=t.parentNode,s=[e],l=[t];if(e===t)return u=!0,0;if(!o||!a)return e===n?-1:t===n?1:o?-1:a?1:0;if(o===a)return ut(e,t);r=e;while(r=r.parentNode)s.unshift(r);r=t;while(r=r.parentNode)l.unshift(r);while(s[i]===l[i])i++;return i?ut(s[i],l[i]):s[i]===w?-1:l[i]===w?1:0},u=!1,[0,0].sort(v),T.detectDuplicates=u,p):p},st.matches=function(e,t){return st(e,null,null,t)},st.matchesSelector=function(e,t){if((e.ownerDocument||e)!==p&&c(e),t=t.replace(Z,"='$1']"),!(!T.matchesSelector||d||g&&g.test(t)||h.test(t)))try{var n=m.call(e,t);if(n||T.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(r){}return st(t,p,null,[e]).length>0},st.contains=function(e,t){return(e.ownerDocument||e)!==p&&c(e),y(e,t)},st.attr=function(e,t){var n;return(e.ownerDocument||e)!==p&&c(e),d||(t=t.toLowerCase()),(n=i.attrHandle[t])?n(e):d||T.attributes?e.getAttribute(t):((n=e.getAttributeNode(t))||e.getAttribute(t))&&e[t]===!0?t:n&&n.specified?n.value:null},st.error=function(e){throw Error("Syntax error, unrecognized expression: "+e)},st.uniqueSort=function(e){var t,n=[],r=1,i=0;if(u=!T.detectDuplicates,e.sort(v),u){for(;t=e[r];r++)t===e[r-1]&&(i=n.push(r));while(i--)e.splice(n[i],1)}return e};function ut(e,t){var n=t&&e,r=n&&(~t.sourceIndex||j)-(~e.sourceIndex||j);if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function lt(e){return function(t){var n=t.nodeName.toLowerCase();return"input"===n&&t.type===e}}function ct(e){return function(t){var n=t.nodeName.toLowerCase();return("input"===n||"button"===n)&&t.type===e}}function pt(e){return ot(function(t){return t=+t,ot(function(n,r){var i,o=e([],n.length,t),a=o.length;while(a--)n[i=o[a]]&&(n[i]=!(r[i]=n[i]))})})}o=st.getText=function(e){var t,n="",r=0,i=e.nodeType;if(i){if(1===i||9===i||11===i){if("string"==typeof e.textContent)return e.textContent;for(e=e.firstChild;e;e=e.nextSibling)n+=o(e)}else if(3===i||4===i)return e.nodeValue}else for(;t=e[r];r++)n+=o(t);return n},i=st.selectors={cacheLength:50,createPseudo:ot,match:U,find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(et,tt),e[3]=(e[4]||e[5]||"").replace(et,tt),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||st.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&st.error(e[0]),e},PSEUDO:function(e){var t,n=!e[5]&&e[2];return U.CHILD.test(e[0])?null:(e[4]?e[2]=e[4]:n&&z.test(n)&&(t=ft(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){return"*"===e?function(){return!0}:(e=e.replace(et,tt).toLowerCase(),function(t){return t.nodeName&&t.nodeName.toLowerCase()===e})},CLASS:function(e){var t=k[e+" "];return t||(t=RegExp("(^|"+_+")"+e+"("+_+"|$)"))&&k(e,function(e){return t.test(e.className||typeof e.getAttribute!==A&&e.getAttribute("class")||"")})},ATTR:function(e,t,n){return function(r){var i=st.attr(r,e);return null==i?"!="===t:t?(i+="","="===t?i===n:"!="===t?i!==n:"^="===t?n&&0===i.indexOf(n):"*="===t?n&&i.indexOf(n)>-1:"$="===t?n&&i.slice(-n.length)===n:"~="===t?(" "+i+" ").indexOf(n)>-1:"|="===t?i===n||i.slice(0,n.length+1)===n+"-":!1):!0}},CHILD:function(e,t,n,r,i){var o="nth"!==e.slice(0,3),a="last"!==e.slice(-4),s="of-type"===t;return 1===r&&0===i?function(e){return!!e.parentNode}:function(t,n,u){var l,c,p,f,d,h,g=o!==a?"nextSibling":"previousSibling",m=t.parentNode,y=s&&t.nodeName.toLowerCase(),v=!u&&!s;if(m){if(o){while(g){p=t;while(p=p[g])if(s?p.nodeName.toLowerCase()===y:1===p.nodeType)return!1;h=g="only"===e&&!h&&"nextSibling"}return!0}if(h=[a?m.firstChild:m.lastChild],a&&v){c=m[x]||(m[x]={}),l=c[e]||[],d=l[0]===N&&l[1],f=l[0]===N&&l[2],p=d&&m.childNodes[d];while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if(1===p.nodeType&&++f&&p===t){c[e]=[N,d,f];break}}else if(v&&(l=(t[x]||(t[x]={}))[e])&&l[0]===N)f=l[1];else while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if((s?p.nodeName.toLowerCase()===y:1===p.nodeType)&&++f&&(v&&((p[x]||(p[x]={}))[e]=[N,f]),p===t))break;return f-=i,f===r||0===f%r&&f/r>=0}}},PSEUDO:function(e,t){var n,r=i.pseudos[e]||i.setFilters[e.toLowerCase()]||st.error("unsupported pseudo: "+e);return r[x]?r(t):r.length>1?(n=[e,e,"",t],i.setFilters.hasOwnProperty(e.toLowerCase())?ot(function(e,n){var i,o=r(e,t),a=o.length;while(a--)i=M.call(e,o[a]),e[i]=!(n[i]=o[a])}):function(e){return r(e,0,n)}):r}},pseudos:{not:ot(function(e){var t=[],n=[],r=s(e.replace(W,"$1"));return r[x]?ot(function(e,t,n,i){var o,a=r(e,null,i,[]),s=e.length;while(s--)(o=a[s])&&(e[s]=!(t[s]=o))}):function(e,i,o){return t[0]=e,r(t,null,o,n),!n.pop()}}),has:ot(function(e){return function(t){return st(e,t).length>0}}),contains:ot(function(e){return function(t){return(t.textContent||t.innerText||o(t)).indexOf(e)>-1}}),lang:ot(function(e){return X.test(e||"")||st.error("unsupported lang: "+e),e=e.replace(et,tt).toLowerCase(),function(t){var n;do if(n=d?t.getAttribute("xml:lang")||t.getAttribute("lang"):t.lang)return n=n.toLowerCase(),n===e||0===n.indexOf(e+"-");while((t=t.parentNode)&&1===t.nodeType);return!1}}),target:function(t){var n=e.location&&e.location.hash;return n&&n.slice(1)===t.id},root:function(e){return e===f},focus:function(e){return e===p.activeElement&&(!p.hasFocus||p.hasFocus())&&!!(e.type||e.href||~e.tabIndex)},enabled:function(e){return e.disabled===!1},disabled:function(e){return e.disabled===!0},checked:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&!!e.checked||"option"===t&&!!e.selected},selected:function(e){return e.parentNode&&e.parentNode.selectedIndex,e.selected===!0},empty:function(e){for(e=e.firstChild;e;e=e.nextSibling)if(e.nodeName>"@"||3===e.nodeType||4===e.nodeType)return!1;return!0},parent:function(e){return!i.pseudos.empty(e)},header:function(e){return Q.test(e.nodeName)},input:function(e){return G.test(e.nodeName)},button:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&"button"===e.type||"button"===t},text:function(e){var t;return"input"===e.nodeName.toLowerCase()&&"text"===e.type&&(null==(t=e.getAttribute("type"))||t.toLowerCase()===e.type)},first:pt(function(){return[0]}),last:pt(function(e,t){return[t-1]}),eq:pt(function(e,t,n){return[0>n?n+t:n]}),even:pt(function(e,t){var n=0;for(;t>n;n+=2)e.push(n);return e}),odd:pt(function(e,t){var n=1;for(;t>n;n+=2)e.push(n);return e}),lt:pt(function(e,t,n){var r=0>n?n+t:n;for(;--r>=0;)e.push(r);return e}),gt:pt(function(e,t,n){var r=0>n?n+t:n;for(;t>++r;)e.push(r);return e})}};for(n in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})i.pseudos[n]=lt(n);for(n in{submit:!0,reset:!0})i.pseudos[n]=ct(n);function ft(e,t){var n,r,o,a,s,u,l,c=E[e+" "];if(c)return t?0:c.slice(0);s=e,u=[],l=i.preFilter;while(s){(!n||(r=$.exec(s)))&&(r&&(s=s.slice(r[0].length)||s),u.push(o=[])),n=!1,(r=I.exec(s))&&(n=r.shift(),o.push({value:n,type:r[0].replace(W," ")}),s=s.slice(n.length));for(a in i.filter)!(r=U[a].exec(s))||l[a]&&!(r=l[a](r))||(n=r.shift(),o.push({value:n,type:a,matches:r}),s=s.slice(n.length));if(!n)break}return t?s.length:s?st.error(e):E(e,u).slice(0)}function dt(e){var t=0,n=e.length,r="";for(;n>t;t++)r+=e[t].value;return r}function ht(e,t,n){var i=t.dir,o=n&&"parentNode"===i,a=C++;return t.first?function(t,n,r){while(t=t[i])if(1===t.nodeType||o)return e(t,n,r)}:function(t,n,s){var u,l,c,p=N+" "+a;if(s){while(t=t[i])if((1===t.nodeType||o)&&e(t,n,s))return!0}else while(t=t[i])if(1===t.nodeType||o)if(c=t[x]||(t[x]={}),(l=c[i])&&l[0]===p){if((u=l[1])===!0||u===r)return u===!0}else if(l=c[i]=[p],l[1]=e(t,n,s)||r,l[1]===!0)return!0}}function gt(e){return e.length>1?function(t,n,r){var i=e.length;while(i--)if(!e[i](t,n,r))return!1;return!0}:e[0]}function mt(e,t,n,r,i){var o,a=[],s=0,u=e.length,l=null!=t;for(;u>s;s++)(o=e[s])&&(!n||n(o,r,i))&&(a.push(o),l&&t.push(s));return a}function yt(e,t,n,r,i,o){return r&&!r[x]&&(r=yt(r)),i&&!i[x]&&(i=yt(i,o)),ot(function(o,a,s,u){var l,c,p,f=[],d=[],h=a.length,g=o||xt(t||"*",s.nodeType?[s]:s,[]),m=!e||!o&&t?g:mt(g,f,e,s,u),y=n?i||(o?e:h||r)?[]:a:m;if(n&&n(m,y,s,u),r){l=mt(y,d),r(l,[],s,u),c=l.length;while(c--)(p=l[c])&&(y[d[c]]=!(m[d[c]]=p))}if(o){if(i||e){if(i){l=[],c=y.length;while(c--)(p=y[c])&&l.push(m[c]=p);i(null,y=[],l,u)}c=y.length;while(c--)(p=y[c])&&(l=i?M.call(o,p):f[c])>-1&&(o[l]=!(a[l]=p))}}else y=mt(y===a?y.splice(h,y.length):y),i?i(null,a,y,u):H.apply(a,y)})}function vt(e){var t,n,r,o=e.length,a=i.relative[e[0].type],s=a||i.relative[" "],u=a?1:0,c=ht(function(e){return e===t},s,!0),p=ht(function(e){return M.call(t,e)>-1},s,!0),f=[function(e,n,r){return!a&&(r||n!==l)||((t=n).nodeType?c(e,n,r):p(e,n,r))}];for(;o>u;u++)if(n=i.relative[e[u].type])f=[ht(gt(f),n)];else{if(n=i.filter[e[u].type].apply(null,e[u].matches),n[x]){for(r=++u;o>r;r++)if(i.relative[e[r].type])break;return yt(u>1&&gt(f),u>1&&dt(e.slice(0,u-1)).replace(W,"$1"),n,r>u&&vt(e.slice(u,r)),o>r&&vt(e=e.slice(r)),o>r&&dt(e))}f.push(n)}return gt(f)}function bt(e,t){var n=0,o=t.length>0,a=e.length>0,s=function(s,u,c,f,d){var h,g,m,y=[],v=0,b="0",x=s&&[],w=null!=d,T=l,C=s||a&&i.find.TAG("*",d&&u.parentNode||u),k=N+=null==T?1:Math.random()||.1;for(w&&(l=u!==p&&u,r=n);null!=(h=C[b]);b++){if(a&&h){g=0;while(m=e[g++])if(m(h,u,c)){f.push(h);break}w&&(N=k,r=++n)}o&&((h=!m&&h)&&v--,s&&x.push(h))}if(v+=b,o&&b!==v){g=0;while(m=t[g++])m(x,y,u,c);if(s){if(v>0)while(b--)x[b]||y[b]||(y[b]=L.call(f));y=mt(y)}H.apply(f,y),w&&!s&&y.length>0&&v+t.length>1&&st.uniqueSort(f)}return w&&(N=k,l=T),x};return o?ot(s):s}s=st.compile=function(e,t){var n,r=[],i=[],o=S[e+" "];if(!o){t||(t=ft(e)),n=t.length;while(n--)o=vt(t[n]),o[x]?r.push(o):i.push(o);o=S(e,bt(i,r))}return o};function xt(e,t,n){var r=0,i=t.length;for(;i>r;r++)st(e,t[r],n);return n}function wt(e,t,n,r){var o,a,u,l,c,p=ft(e);if(!r&&1===p.length){if(a=p[0]=p[0].slice(0),a.length>2&&"ID"===(u=a[0]).type&&9===t.nodeType&&!d&&i.relative[a[1].type]){if(t=i.find.ID(u.matches[0].replace(et,tt),t)[0],!t)return n;e=e.slice(a.shift().value.length)}o=U.needsContext.test(e)?0:a.length;while(o--){if(u=a[o],i.relative[l=u.type])break;if((c=i.find[l])&&(r=c(u.matches[0].replace(et,tt),V.test(a[0].type)&&t.parentNode||t))){if(a.splice(o,1),e=r.length&&dt(a),!e)return H.apply(n,q.call(r,0)),n;break}}}return s(e,p)(r,t,d,n,V.test(e)),n}i.pseudos.nth=i.pseudos.eq;function Tt(){}i.filters=Tt.prototype=i.pseudos,i.setFilters=new Tt,c(),st.attr=b.attr,b.find=st,b.expr=st.selectors,b.expr[":"]=b.expr.pseudos,b.unique=st.uniqueSort,b.text=st.getText,b.isXMLDoc=st.isXML,b.contains=st.contains}(e);var at=/Until$/,st=/^(?:parents|prev(?:Until|All))/,ut=/^.[^:#\[\.,]*$/,lt=b.expr.match.needsContext,ct={children:!0,contents:!0,next:!0,prev:!0};b.fn.extend({find:function(e){var t,n,r,i=this.length;if("string"!=typeof e)return r=this,this.pushStack(b(e).filter(function(){for(t=0;i>t;t++)if(b.contains(r[t],this))return!0}));for(n=[],t=0;i>t;t++)b.find(e,this[t],n);return n=this.pushStack(i>1?b.unique(n):n),n.selector=(this.selector?this.selector+" ":"")+e,n},has:function(e){var t,n=b(e,this),r=n.length;return this.filter(function(){for(t=0;r>t;t++)if(b.contains(this,n[t]))return!0})},not:function(e){return this.pushStack(ft(this,e,!1))},filter:function(e){return this.pushStack(ft(this,e,!0))},is:function(e){return!!e&&("string"==typeof e?lt.test(e)?b(e,this.context).index(this[0])>=0:b.filter(e,this).length>0:this.filter(e).length>0)},closest:function(e,t){var n,r=0,i=this.length,o=[],a=lt.test(e)||"string"!=typeof e?b(e,t||this.context):0;for(;i>r;r++){n=this[r];while(n&&n.ownerDocument&&n!==t&&11!==n.nodeType){if(a?a.index(n)>-1:b.find.matchesSelector(n,e)){o.push(n);break}n=n.parentNode}}return this.pushStack(o.length>1?b.unique(o):o)},index:function(e){return e?"string"==typeof e?b.inArray(this[0],b(e)):b.inArray(e.jquery?e[0]:e,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(e,t){var n="string"==typeof e?b(e,t):b.makeArray(e&&e.nodeType?[e]:e),r=b.merge(this.get(),n);return this.pushStack(b.unique(r))},addBack:function(e){return this.add(null==e?this.prevObject:this.prevObject.filter(e))}}),b.fn.andSelf=b.fn.addBack;function pt(e,t){do e=e[t];while(e&&1!==e.nodeType);return e}b.each({parent:function(e){var t=e.parentNode;return t&&11!==t.nodeType?t:null},parents:function(e){return b.dir(e,"parentNode")},parentsUntil:function(e,t,n){return b.dir(e,"parentNode",n)},next:function(e){return pt(e,"nextSibling")},prev:function(e){return pt(e,"previousSibling")},nextAll:function(e){return b.dir(e,"nextSibling")},prevAll:function(e){return b.dir(e,"previousSibling")},nextUntil:function(e,t,n){return b.dir(e,"nextSibling",n)},prevUntil:function(e,t,n){return b.dir(e,"previousSibling",n)},siblings:function(e){return b.sibling((e.parentNode||{}).firstChild,e)},children:function(e){return b.sibling(e.firstChild)},contents:function(e){return b.nodeName(e,"iframe")?e.contentDocument||e.contentWindow.document:b.merge([],e.childNodes)}},function(e,t){b.fn[e]=function(n,r){var i=b.map(this,t,n);return at.test(e)||(r=n),r&&"string"==typeof r&&(i=b.filter(r,i)),i=this.length>1&&!ct[e]?b.unique(i):i,this.length>1&&st.test(e)&&(i=i.reverse()),this.pushStack(i)}}),b.extend({filter:function(e,t,n){return n&&(e=":not("+e+")"),1===t.length?b.find.matchesSelector(t[0],e)?[t[0]]:[]:b.find.matches(e,t)},dir:function(e,n,r){var i=[],o=e[n];while(o&&9!==o.nodeType&&(r===t||1!==o.nodeType||!b(o).is(r)))1===o.nodeType&&i.push(o),o=o[n];return i},sibling:function(e,t){var n=[];for(;e;e=e.nextSibling)1===e.nodeType&&e!==t&&n.push(e);return n}});function ft(e,t,n){if(t=t||0,b.isFunction(t))return b.grep(e,function(e,r){var i=!!t.call(e,r,e);return i===n});if(t.nodeType)return b.grep(e,function(e){return e===t===n});if("string"==typeof t){var r=b.grep(e,function(e){return 1===e.nodeType});if(ut.test(t))return b.filter(t,r,!n);t=b.filter(t,r)}return b.grep(e,function(e){return b.inArray(e,t)>=0===n})}function dt(e){var t=ht.split("|"),n=e.createDocumentFragment();if(n.createElement)while(t.length)n.createElement(t.pop());return n}var ht="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",gt=/ jQuery\d+="(?:null|\d+)"/g,mt=RegExp("<(?:"+ht+")[\\s/>]","i"),yt=/^\s+/,vt=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bt=/<([\w:]+)/,xt=/<tbody/i,wt=/<|&#?\w+;/,Tt=/<(?:script|style|link)/i,Nt=/^(?:checkbox|radio)$/i,Ct=/checked\s*(?:[^=]|=\s*.checked.)/i,kt=/^$|\/(?:java|ecma)script/i,Et=/^true\/(.*)/,St=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,At={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:b.support.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},jt=dt(o),Dt=jt.appendChild(o.createElement("div"));At.optgroup=At.option,At.tbody=At.tfoot=At.colgroup=At.caption=At.thead,At.th=At.td,b.fn.extend({text:function(e){return b.access(this,function(e){return e===t?b.text(this):this.empty().append((this[0]&&this[0].ownerDocument||o).createTextNode(e))},null,e,arguments.length)},wrapAll:function(e){if(b.isFunction(e))return this.each(function(t){b(this).wrapAll(e.call(this,t))});if(this[0]){var t=b(e,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&t.insertBefore(this[0]),t.map(function(){var e=this;while(e.firstChild&&1===e.firstChild.nodeType)e=e.firstChild;return e}).append(this)}return this},wrapInner:function(e){return b.isFunction(e)?this.each(function(t){b(this).wrapInner(e.call(this,t))}):this.each(function(){var t=b(this),n=t.contents();n.length?n.wrapAll(e):t.append(e)})},wrap:function(e){var t=b.isFunction(e);return this.each(function(n){b(this).wrapAll(t?e.call(this,n):e)})},unwrap:function(){return this.parent().each(function(){b.nodeName(this,"body")||b(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.appendChild(e)})},prepend:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.insertBefore(e,this.firstChild)})},before:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this)})},after:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this.nextSibling)})},remove:function(e,t){var n,r=0;for(;null!=(n=this[r]);r++)(!e||b.filter(e,[n]).length>0)&&(t||1!==n.nodeType||b.cleanData(Ot(n)),n.parentNode&&(t&&b.contains(n.ownerDocument,n)&&Mt(Ot(n,"script")),n.parentNode.removeChild(n)));return this},empty:function(){var e,t=0;for(;null!=(e=this[t]);t++){1===e.nodeType&&b.cleanData(Ot(e,!1));while(e.firstChild)e.removeChild(e.firstChild);e.options&&b.nodeName(e,"select")&&(e.options.length=0)}return this},clone:function(e,t){return e=null==e?!1:e,t=null==t?e:t,this.map(function(){return b.clone(this,e,t)})},html:function(e){return b.access(this,function(e){var n=this[0]||{},r=0,i=this.length;if(e===t)return 1===n.nodeType?n.innerHTML.replace(gt,""):t;if(!("string"!=typeof e||Tt.test(e)||!b.support.htmlSerialize&&mt.test(e)||!b.support.leadingWhitespace&&yt.test(e)||At[(bt.exec(e)||["",""])[1].toLowerCase()])){e=e.replace(vt,"<$1></$2>");try{for(;i>r;r++)n=this[r]||{},1===n.nodeType&&(b.cleanData(Ot(n,!1)),n.innerHTML=e);n=0}catch(o){}}n&&this.empty().append(e)},null,e,arguments.length)},replaceWith:function(e){var t=b.isFunction(e);return t||"string"==typeof e||(e=b(e).not(this).detach()),this.domManip([e],!0,function(e){var t=this.nextSibling,n=this.parentNode;n&&(b(this).remove(),n.insertBefore(e,t))})},detach:function(e){return this.remove(e,!0)},domManip:function(e,n,r){e=f.apply([],e);var i,o,a,s,u,l,c=0,p=this.length,d=this,h=p-1,g=e[0],m=b.isFunction(g);if(m||!(1>=p||"string"!=typeof g||b.support.checkClone)&&Ct.test(g))return this.each(function(i){var o=d.eq(i);m&&(e[0]=g.call(this,i,n?o.html():t)),o.domManip(e,n,r)});if(p&&(l=b.buildFragment(e,this[0].ownerDocument,!1,this),i=l.firstChild,1===l.childNodes.length&&(l=i),i)){for(n=n&&b.nodeName(i,"tr"),s=b.map(Ot(l,"script"),Ht),a=s.length;p>c;c++)o=l,c!==h&&(o=b.clone(o,!0,!0),a&&b.merge(s,Ot(o,"script"))),r.call(n&&b.nodeName(this[c],"table")?Lt(this[c],"tbody"):this[c],o,c);if(a)for(u=s[s.length-1].ownerDocument,b.map(s,qt),c=0;a>c;c++)o=s[c],kt.test(o.type||"")&&!b._data(o,"globalEval")&&b.contains(u,o)&&(o.src?b.ajax({url:o.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):b.globalEval((o.text||o.textContent||o.innerHTML||"").replace(St,"")));l=i=null}return this}});function Lt(e,t){return e.getElementsByTagName(t)[0]||e.appendChild(e.ownerDocument.createElement(t))}function Ht(e){var t=e.getAttributeNode("type");return e.type=(t&&t.specified)+"/"+e.type,e}function qt(e){var t=Et.exec(e.type);return t?e.type=t[1]:e.removeAttribute("type"),e}function Mt(e,t){var n,r=0;for(;null!=(n=e[r]);r++)b._data(n,"globalEval",!t||b._data(t[r],"globalEval"))}function _t(e,t){if(1===t.nodeType&&b.hasData(e)){var n,r,i,o=b._data(e),a=b._data(t,o),s=o.events;if(s){delete a.handle,a.events={};for(n in s)for(r=0,i=s[n].length;i>r;r++)b.event.add(t,n,s[n][r])}a.data&&(a.data=b.extend({},a.data))}}function Ft(e,t){var n,r,i;if(1===t.nodeType){if(n=t.nodeName.toLowerCase(),!b.support.noCloneEvent&&t[b.expando]){i=b._data(t);for(r in i.events)b.removeEvent(t,r,i.handle);t.removeAttribute(b.expando)}"script"===n&&t.text!==e.text?(Ht(t).text=e.text,qt(t)):"object"===n?(t.parentNode&&(t.outerHTML=e.outerHTML),b.support.html5Clone&&e.innerHTML&&!b.trim(t.innerHTML)&&(t.innerHTML=e.innerHTML)):"input"===n&&Nt.test(e.type)?(t.defaultChecked=t.checked=e.checked,t.value!==e.value&&(t.value=e.value)):"option"===n?t.defaultSelected=t.selected=e.defaultSelected:("input"===n||"textarea"===n)&&(t.defaultValue=e.defaultValue)}}b.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(e,t){b.fn[e]=function(e){var n,r=0,i=[],o=b(e),a=o.length-1;for(;a>=r;r++)n=r===a?this:this.clone(!0),b(o[r])[t](n),d.apply(i,n.get());return this.pushStack(i)}});function Ot(e,n){var r,o,a=0,s=typeof e.getElementsByTagName!==i?e.getElementsByTagName(n||"*"):typeof e.querySelectorAll!==i?e.querySelectorAll(n||"*"):t;if(!s)for(s=[],r=e.childNodes||e;null!=(o=r[a]);a++)!n||b.nodeName(o,n)?s.push(o):b.merge(s,Ot(o,n));return n===t||n&&b.nodeName(e,n)?b.merge([e],s):s}function Bt(e){Nt.test(e.type)&&(e.defaultChecked=e.checked)}b.extend({clone:function(e,t,n){var r,i,o,a,s,u=b.contains(e.ownerDocument,e);if(b.support.html5Clone||b.isXMLDoc(e)||!mt.test("<"+e.nodeName+">")?o=e.cloneNode(!0):(Dt.innerHTML=e.outerHTML,Dt.removeChild(o=Dt.firstChild)),!(b.support.noCloneEvent&&b.support.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||b.isXMLDoc(e)))for(r=Ot(o),s=Ot(e),a=0;null!=(i=s[a]);++a)r[a]&&Ft(i,r[a]);if(t)if(n)for(s=s||Ot(e),r=r||Ot(o),a=0;null!=(i=s[a]);a++)_t(i,r[a]);else _t(e,o);return r=Ot(o,"script"),r.length>0&&Mt(r,!u&&Ot(e,"script")),r=s=i=null,o},buildFragment:function(e,t,n,r){var i,o,a,s,u,l,c,p=e.length,f=dt(t),d=[],h=0;for(;p>h;h++)if(o=e[h],o||0===o)if("object"===b.type(o))b.merge(d,o.nodeType?[o]:o);else if(wt.test(o)){s=s||f.appendChild(t.createElement("div")),u=(bt.exec(o)||["",""])[1].toLowerCase(),c=At[u]||At._default,s.innerHTML=c[1]+o.replace(vt,"<$1></$2>")+c[2],i=c[0];while(i--)s=s.lastChild;if(!b.support.leadingWhitespace&&yt.test(o)&&d.push(t.createTextNode(yt.exec(o)[0])),!b.support.tbody){o="table"!==u||xt.test(o)?"<table>"!==c[1]||xt.test(o)?0:s:s.firstChild,i=o&&o.childNodes.length;while(i--)b.nodeName(l=o.childNodes[i],"tbody")&&!l.childNodes.length&&o.removeChild(l)
-}b.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),b.support.appendChecked||b.grep(Ot(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===b.inArray(o,r))&&(a=b.contains(o.ownerDocument,o),s=Ot(f.appendChild(o),"script"),a&&Mt(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,u=b.expando,l=b.cache,p=b.support.deleteExpando,f=b.event.special;for(;null!=(n=e[s]);s++)if((t||b.acceptData(n))&&(o=n[u],a=o&&l[o])){if(a.events)for(r in a.events)f[r]?b.event.remove(n,r):b.removeEvent(n,r,a.handle);l[o]&&(delete l[o],p?delete n[u]:typeof n.removeAttribute!==i?n.removeAttribute(u):n[u]=null,c.push(o))}}});var Pt,Rt,Wt,$t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+x+")(.*)$","i"),Yt=RegExp("^("+x+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+x+")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===b.css(e,"display")||!b.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=b._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=b._data(r,"olddisplay",un(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&b._data(r,"olddisplay",i?n:b.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style&&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}b.fn.extend({css:function(e,n){return b.access(this,function(e,n,r){var i,o,a={},s=0;if(b.isArray(n)){for(o=Rt(e),i=n.length;i>s;s++)a[n[s]]=b.css(e,n[s],!1,o);return a}return r!==t?b.style(e,n,r):b.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?b(this).show():b(this).hide()})}}),b.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":b.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,u=b.camelCase(n),l=e.style;if(n=b.cssProps[u]||(b.cssProps[u]=tn(l,u)),s=b.cssHooks[n]||b.cssHooks[u],r===t)return s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:l[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(b.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||b.cssNumber[u]||(r+="px"),b.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(l[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{l[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,u=b.camelCase(n);return n=b.cssProps[u]||(b.cssProps[u]=tn(e.style,u)),s=b.cssHooks[n]||b.cssHooks[u],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||b.isNumeric(o)?o||0:a):a},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply(e,r||[]);for(o in t)e.style[o]=a[o];return i}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s.getPropertyValue(n)||s[n]:t,l=e.style;return s&&(""!==u||b.contains(e.ownerDocument,e)||(u=b.style(e,n)),Yt.test(u)&&Ut.test(n)&&(i=l.width,o=l.minWidth,a=l.maxWidth,l.minWidth=l.maxWidth=l.width=u,u=s.width,l.width=i,l.minWidth=o,l.maxWidth=a)),u}):o.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s[n]:t,l=e.style;return null==u&&l&&l[n]&&(u=l[n]),Yt.test(u)&&!zt.test(n)&&(i=l.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),l.left="fontSize"===n?"1em":u,u=l.pixelLeft+"px",l.left=i,a&&(o.left=a)),""===u?"auto":u});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=b.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=b.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=b.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=b.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=b.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style[t]),Yt.test(i))return i;r=a&&(b.support.boxSizingReliable||i===e.style[t]),i=parseFloat(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function un(e){var t=o,n=Gt[e];return n||(n=ln(e,t),"none"!==n&&n||(Pt=(Pt||b("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=ln(e,t),Pt.detach()),Gt[e]=n),n}function ln(e,t){var n=b(t.createElement(e)).appendTo(t.body),r=b.css(n[0],"display");return n.remove(),r}b.each(["height","width"],function(e,n){b.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(b.css(e,"display"))?b.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,i),i):0)}}}),b.support.opacity||(b.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentStyle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":""},set:function(e,t){var n=e.style,r=e.currentStyle,i=b.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===b.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),b(function(){b.support.reliableMarginRight||(b.cssHooks.marginRight={get:function(e,n){return n?b.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!b.support.pixelPosition&&b.fn.position&&b.each(["top","left"],function(e,n){b.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?b(e).position()[n]+"px":r):t}}})}),b.expr&&b.expr.filters&&(b.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!b.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||b.css(e,"display"))},b.expr.filters.visible=function(e){return!b.expr.filters.hidden(e)}),b.each({margin:"",padding:"",border:"Width"},function(e,t){b.cssHooks[e+t]={expand:function(n){var r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(b.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;b.fn.extend({serialize:function(){return b.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=b.prop(this,"elements");return e?b.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!b(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Nt.test(e))}).map(function(e,t){var n=b(this).val();return null==n?null:b.isArray(n)?b.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),b.param=function(e,n){var r,i=[],o=function(e,t){t=b.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encodeURIComponent(t)};if(n===t&&(n=b.ajaxSettings&&b.ajaxSettings.traditional),b.isArray(e)||e.jquery&&!b.isPlainObject(e))b.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(b.isArray(t))b.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==b.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}b.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){b.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),b.fn.hover=function(e,t){return this.mouseenter(e).mouseleave(t||e)};var mn,yn,vn=b.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Nn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Cn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=b.fn.load,An={},jn={},Dn="*/".concat("*");try{yn=a.href}catch(Ln){yn=o.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(w)||[];if(b.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(u){var l;return o[u]=!0,b.each(e[u]||[],function(e,u){var c=u(n,r,i);return"string"!=typeof c||a||o[c]?a?!(l=c):t:(n.dataTypes.unshift(c),s(c),!1)}),l}return s(n.dataTypes[0])||!o["*"]&&s("*")}function Mn(e,n){var r,i,o=b.ajaxSettings.flatOptions||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&b.extend(!0,e,r),e}b.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,u=e.indexOf(" ");return u>=0&&(i=e.slice(u,e.length),e=e.slice(0,u)),b.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&b.ajax({url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?b("<div>").append(b.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},b.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){b.fn[t]=function(e){return this.on(t,e)}}),b.each(["get","post"],function(e,n){b[n]=function(e,r,i,o){return b.isFunction(r)&&(o=o||i,i=r,r=t),b.ajax({url:e,type:n,dataType:o,data:r,success:i})}}),b.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Nn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":e.String,"text html":!0,"text json":b.parseJSON,"text xml":b.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?Mn(Mn(e,b.ajaxSettings),t):Mn(b.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,u,l,c,p=b.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?b(f):b.event,h=b.Deferred(),g=b.Callbacks("once memory"),m=p.statusCode||{},y={},v={},x=0,T="canceled",N={readyState:0,getResponseHeader:function(e){var t;if(2===x){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===x?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return x||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return x||(p.mimeType=e),this},statusCode:function(e){var t;if(e)if(2>x)for(t in e)m[t]=[m[t],e[t]];else N.always(e[N.status]);return this},abort:function(e){var t=e||T;return l&&l.abort(t),k(0,t),this}};if(h.promise(N).complete=g.add,N.success=N.done,N.error=N.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=b.trim(p.dataType||"*").toLowerCase().match(w)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?80:443))==(mn[3]||("http:"===mn[1]?80:443)))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=b.param(p.data,p.traditional)),qn(An,p,n,N),2===x)return N;u=p.global,u&&0===b.active++&&b.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Cn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(b.lastModified[o]&&N.setRequestHeader("If-Modified-Since",b.lastModified[o]),b.etag[o]&&N.setRequestHeader("If-None-Match",b.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&N.setRequestHeader("Content-Type",p.contentType),N.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)N.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,N,p)===!1||2===x))return N.abort();T="abort";for(i in{success:1,error:1,complete:1})N[i](p[i]);if(l=qn(jn,p,n,N)){N.readyState=1,u&&d.trigger("ajaxSend",[N,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){N.abort("timeout")},p.timeout));try{x=1,l.send(y,k)}catch(C){if(!(2>x))throw C;k(-1,C)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,C=n;2!==x&&(x=2,s&&clearTimeout(s),l=t,a=i||"",N.readyState=e>0?4:0,r&&(w=_n(p,N,r)),e>=200&&300>e||304===e?(p.ifModified&&(T=N.getResponseHeader("Last-Modified"),T&&(b.lastModified[o]=T),T=N.getResponseHeader("etag"),T&&(b.etag[o]=T)),204===e?(c=!0,C="nocontent"):304===e?(c=!0,C="notmodified"):(c=Fn(p,w),C=c.state,y=c.data,v=c.error,c=!v)):(v=C,(e||!C)&&(C="error",0>e&&(e=0))),N.status=e,N.statusText=(n||C)+"",c?h.resolveWith(f,[y,C,N]):h.rejectWith(f,[N,C,v]),N.statusCode(m),m=t,u&&d.trigger(c?"ajaxSuccess":"ajaxError",[N,p,c?y:v]),g.fireWith(f,[N,C]),u&&(d.trigger("ajaxComplete",[N,p]),--b.active||b.event.trigger("ajaxStop")))}return N},getScript:function(e,n){return b.get(e,t,n,"script")},getJSON:function(e,t,n){return b.get(e,t,n,"json")}});function _n(e,n,r){var i,o,a,s,u=e.contents,l=e.dataTypes,c=e.responseFields;for(s in c)s in r&&(n[c[s]]=r[s]);while("*"===l[0])l.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in u)if(u[s]&&u[s].test(o)){l.unshift(s);break}if(l[0]in r)a=l[0];else{for(s in r){if(!l[0]||e.converters[s+" "+l[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==l[0]&&l.unshift(a),r[a]):t}function Fn(e,t){var n,r,i,o,a={},s=0,u=e.dataTypes.slice(),l=u[0];if(e.dataFilter&&(t=e.dataFilter(t,e.dataType)),u[1])for(i in e.converters)a[i.toLowerCase()]=e.converters[i];for(;r=u[++s];)if("*"!==r){if("*"!==l&&l!==r){if(i=a[l+" "+r]||a["* "+r],!i)for(n in a)if(o=n.split(" "),o[1]===r&&(i=a[l+" "+o[0]]||a["* "+o[0]])){i===!0?i=a[n]:a[n]!==!0&&(r=o[0],u.splice(s--,0,r));break}if(i!==!0)if(i&&e["throws"])t=i(t);else try{t=i(t)}catch(c){return{state:"parsererror",error:i?c:"No conversion from "+l+" to "+r}}}l=r}return{state:"success",data:t}}b.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return b.globalEval(e),e}}}),b.ajaxPrefilter("script",function(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),b.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=o.head||b("head")[0]||o.documentElement;return{send:function(t,i){n=o.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readyState))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var On=[],Bn=/(=)\?(?=&|$)|\?\?/;b.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=On.pop()||b.expando+"_"+vn++;return this[e]=!0,e}}),b.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,u=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return u||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=b.isFunction(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,u?n[u]=n[u].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||b.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,On.push(o)),s&&b.isFunction(a)&&a(s[0]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}b.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=b.ajaxSettings.xhr(),b.support.cors=!!Rn&&"withCredentials"in Rn,Rn=b.support.ajax=!!Rn,Rn&&b.ajaxTransport(function(n){if(!n.crossDomain||b.support.cors){var r;return{send:function(i,o){var a,s,u=n.xhr();if(n.username?u.open(n.type,n.url,n.async,n.username,n.password):u.open(n.type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)u[s]=n.xhrFields[s];n.mimeType&&u.overrideMimeType&&u.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)u.setRequestHeader(s,i[s])}catch(l){}u.send(n.hasContent&&n.data||null),r=function(e,i){var s,l,c,p;try{if(r&&(i||4===u.readyState))if(r=t,a&&(u.onreadystatechange=b.noop,$n&&delete Pn[a]),i)4!==u.readyState&&u.abort();else{p={},s=u.status,l=u.getAllResponseHeaders(),"string"==typeof u.responseText&&(p.text=u.responseText);try{c=u.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,l)},n.async?4===u.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},b(e).unload($n)),Pn[a]=r),u.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+x+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n,r,i=this.createTween(e,t),o=Yn.exec(t),a=i.cur(),s=+a||0,u=1,l=20;if(o){if(n=+o[2],r=o[3]||(b.cssNumber[e]?"":"px"),"px"!==r&&s){s=b.css(i.elem,e,!0)||n||1;do u=u||".5",s/=u,b.style(i.elem,e,s+r);while(u!==(u=i.cur()/a)&&1!==u&&--l)}i.unit=r,i.start=s,i.end=o[1]?s+(o[1]+1)*n:n}return i}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=b.now()}function Zn(e,t){b.each(t,function(t,n){var r=(Qn[t]||[]).concat(Qn["*"]),i=0,o=r.length;for(;o>i;i++)if(r[i].call(e,t,n))return})}function er(e,t,n){var r,i,o=0,a=Gn.length,s=b.Deferred().always(function(){delete u.elem}),u=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,l.startTime+l.duration-t),r=n/l.duration||0,o=1-r,a=0,u=l.tweens.length;for(;u>a;a++)l.tweens[a].run(o);return s.notifyWith(e,[l,o,n]),1>o&&u?n:(s.resolveWith(e,[l]),!1)},l=s.promise({elem:e,props:b.extend({},t),opts:b.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],createTween:function(t,n){var r=b.Tween(e,l.opts,t,n,l.opts.specialEasing[t]||l.opts.easing);return l.tweens.push(r),r},stop:function(t){var n=0,r=t?l.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)l.tweens[n].run(1);return t?s.resolveWith(e,[l,t]):s.rejectWith(e,[l,t]),this}}),c=l.props;for(tr(c,l.opts.specialEasing);a>o;o++)if(r=Gn[o].call(l,e,c,l.opts))return r;return Zn(l,c),b.isFunction(l.opts.start)&&l.opts.start.call(e,l),b.fx.timer(b.extend(u,{elem:e,anim:l,queue:l.opts.queue})),l.progress(l.opts.progress).done(l.opts.done,l.opts.complete).fail(l.opts.fail).always(l.opts.always)}function tr(e,t){var n,r,i,o,a;for(i in e)if(r=b.camelCase(i),o=t[r],n=e[i],b.isArray(n)&&(o=n[1],n=e[i]=n[0]),i!==r&&(e[r]=n,delete e[i]),a=b.cssHooks[r],a&&"expand"in a){n=a.expand(n),delete e[r];for(i in n)i in e||(e[i]=n[i],t[i]=o)}else t[r]=o}b.Animation=b.extend(er,{tweener:function(e,t){b.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,u,l,c,p,f=this,d=e.style,h={},g=[],m=e.nodeType&&nn(e);n.queue||(c=b._queueHooks(e,"fx"),null==c.unqueued&&(c.unqueued=0,p=c.empty.fire,c.empty.fire=function(){c.unqueued||p()}),c.unqueued++,f.always(function(){f.always(function(){c.unqueued--,b.queue(e,"fx").length||c.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overflow=[d.overflow,d.overflowX,d.overflowY],"inline"===b.css(e,"display")&&"none"===b.css(e,"float")&&(b.support.inlineBlockNeedsLayout&&"inline"!==un(e.nodeName)?d.zoom=1:d.display="inline-block")),n.overflow&&(d.overflow="hidden",b.support.shrinkWrapBlocks||f.always(function(){d.overflow=n.overflow[0],d.overflowX=n.overflow[1],d.overflowY=n.overflow[2]}));for(i in t)if(a=t[i],Vn.exec(a)){if(delete t[i],u=u||"toggle"===a,a===(m?"hide":"show"))continue;g.push(i)}if(o=g.length){s=b._data(e,"fxshow")||b._data(e,"fxshow",{}),"hidden"in s&&(m=s.hidden),u&&(s.hidden=!m),m?b(e).show():f.done(function(){b(e).hide()}),f.done(function(){var t;b._removeData(e,"fxshow");for(t in h)b.style(e,t,h[t])});for(i=0;o>i;i++)r=g[i],l=f.createTween(r,m?s[r]:0),h[r]=s[r]||b.style(e,r),r in s||(s[r]=l.start,m&&(l.end=l.start,l.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}b.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this.prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(b.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?b.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototype.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=b.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){b.fx.step[e.prop]?b.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[b.cssProps[e.prop]]||b.cssHooks[e.prop])?b.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scrollTop=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},b.each(["toggle","show","hide"],function(e,t){var n=b.fn[t];b.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),b.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=b.isEmptyObject(e),o=b.speed(t,n,r),a=function(){var t=er(this,b.extend({},e),o);a.finish=function(){t.stop(!0)},(i||b._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.queue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=b.timers,a=b._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].elem!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&b.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=b._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=b.timers,a=r?r.length:0;for(n.finish=!0,b.queue(this,e,[]),i&&i.cur&&i.cur.finish&&i.cur.finish.call(this),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["margin"+n]=r["padding"+n]=e;return t&&(r.opacity=r.width=e),r}b.each({slideDown:ir("show"),slideUp:ir("hide"),slideToggle:ir("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(e,t){b.fn[e]=function(e,n,r){return this.animate(t,e,n,r)}}),b.speed=function(e,t,n){var r=e&&"object"==typeof e?b.extend({},e):{complete:n||!n&&t||b.isFunction(e)&&e,duration:e,easing:n&&t||t&&!b.isFunction(t)&&t};return r.duration=b.fx.off?0:"number"==typeof r.duration?r.duration:r.duration in b.fx.speeds?b.fx.speeds[r.duration]:b.fx.speeds._default,(null==r.queue||r.queue===!0)&&(r.queue="fx"),r.old=r.complete,r.complete=function(){b.isFunction(r.old)&&r.old.call(this),r.queue&&b.dequeue(this,r.queue)},r},b.easing={linear:function(e){return e},swing:function(e){return.5-Math.cos(e*Math.PI)/2}},b.timers=[],b.fx=rr.prototype.init,b.fx.tick=function(){var e,n=b.timers,r=0;for(Xn=b.now();n.length>r;r++)e=n[r],e()||n[r]!==e||n.splice(r--,1);n.length||b.fx.stop(),Xn=t},b.fx.timer=function(e){e()&&b.timers.push(e)&&b.fx.start()},b.fx.interval=13,b.fx.start=function(){Un||(Un=setInterval(b.fx.tick,b.fx.interval))},b.fx.stop=function(){clearInterval(Un),Un=null},b.fx.speeds={slow:600,fast:200,_default:400},b.fx.step={},b.expr&&b.expr.filters&&(b.expr.filters.animated=function(e){return b.grep(b.timers,function(t){return e===t.elem}).length}),b.fn.offset=function(e){if(arguments.length)return e===t?this:this.each(function(t){b.offset.setOffset(this,e,t)});var n,r,o={top:0,left:0},a=this[0],s=a&&a.ownerDocument;if(s)return n=s.documentElement,b.contains(n,a)?(typeof a.getBoundingClientRect!==i&&(o=a.getBoundingClientRect()),r=or(s),{top:o.top+(r.pageYOffset||n.scrollTop)-(n.clientTop||0),left:o.left+(r.pageXOffset||n.scrollLeft)-(n.clientLeft||0)}):o},b.offset={setOffset:function(e,t,n){var r=b.css(e,"position");"static"===r&&(e.style.position="relative");var i=b(e),o=i.offset(),a=b.css(e,"top"),s=b.css(e,"left"),u=("absolute"===r||"fixed"===r)&&b.inArray("auto",[a,s])>-1,l={},c={},p,f;u?(c=i.position(),p=c.top,f=c.left):(p=parseFloat(a)||0,f=parseFloat(s)||0),b.isFunction(t)&&(t=t.call(e,n,o)),null!=t.top&&(l.top=t.top-o.top+p),null!=t.left&&(l.left=t.left-o.left+f),"using"in t?t.using.call(e,l):i.css(l)}},b.fn.extend({position:function(){if(this[0]){var e,t,n={top:0,left:0},r=this[0];return"fixed"===b.css(r,"position")?t=r.getBoundingClientRect():(e=this.offsetParent(),t=this.offset(),b.nodeName(e[0],"html")||(n=e.offset()),n.top+=b.css(e[0],"borderTopWidth",!0),n.left+=b.css(e[0],"borderLeftWidth",!0)),{top:t.top-n.top-b.css(r,"marginTop",!0),left:t.left-n.left-b.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent||o.documentElement;while(e&&!b.nodeName(e,"html")&&"static"===b.css(e,"position"))e=e.offsetParent;return e||o.documentElement})}}),b.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(e,n){var r=/Y/.test(n);b.fn[e]=function(i){return b.access(this,function(e,i,o){var a=or(e);return o===t?a?n in a?a[n]:a.document.documentElement[i]:e[i]:(a?a.scrollTo(r?b(a).scrollLeft():o,r?o:b(a).scrollTop()):e[i]=o,t)},e,i,arguments.length,null)}});function or(e){return b.isWindow(e)?e:9===e.nodeType?e.defaultView||e.parentWindow:!1}b.each({Height:"height",Width:"width"},function(e,n){b.each({padding:"inner"+e,content:n,"":"outer"+e},function(r,i){b.fn[i]=function(i,o){var a=arguments.length&&(r||"boolean"!=typeof i),s=r||(i===!0||o===!0?"margin":"border");return b.access(this,function(n,r,i){var o;return b.isWindow(n)?n.document.documentElement["client"+e]:9===n.nodeType?(o=n.documentElement,Math.max(n.body["scroll"+e],o["scroll"+e],n.body["offset"+e],o["offset"+e],o["client"+e])):i===t?b.css(n,r,s):b.style(n,r,i,s)},n,a?i:t,a,null)}})}),e.jQuery=e.$=b,"function"==typeof define&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return b})})(window);
\ No newline at end of file
+}b.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),b.support.appendChecked||b.grep(Ot(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===b.inArray(o,r))&&(a=b.contains(o.ownerDocument,o),s=Ot(f.appendChild(o),"script"),a&&Mt(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,u=b.expando,l=b.cache,p=b.support.deleteExpando,f=b.event.special;for(;null!=(n=e[s]);s++)if((t||b.acceptData(n))&&(o=n[u],a=o&&l[o])){if(a.events)for(r in a.events)f[r]?b.event.remove(n,r):b.removeEvent(n,r,a.handle);l[o]&&(delete l[o],p?delete n[u]:typeof n.removeAttribute!==i?n.removeAttribute(u):n[u]=null,c.push(o))}}});var Pt,Rt,Wt,$t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+x+")(.*)$","i"),Yt=RegExp("^("+x+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+x+")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===b.css(e,"display")||!b.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=b._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=b._data(r,"olddisplay",un(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&b._data(r,"olddisplay",i?n:b.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style&&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}b.fn.extend({css:function(e,n){return b.access(this,function(e,n,r){var i,o,a={},s=0;if(b.isArray(n)){for(o=Rt(e),i=n.length;i>s;s++)a[n[s]]=b.css(e,n[s],!1,o);return a}return r!==t?b.style(e,n,r):b.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?b(this).show():b(this).hide()})}}),b.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":b.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,u=b.camelCase(n),l=e.style;if(n=b.cssProps[u]||(b.cssProps[u]=tn(l,u)),s=b.cssHooks[n]||b.cssHooks[u],r===t)return s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:l[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(b.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||b.cssNumber[u]||(r+="px"),b.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(l[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{l[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,u=b.camelCase(n);return n=b.cssProps[u]||(b.cssProps[u]=tn(e.style,u)),s=b.cssHooks[n]||b.cssHooks[u],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||b.isNumeric(o)?o||0:a):a},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply(e,r||[]);for(o in t)e.style[o]=a[o];return i}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s.getPropertyValue(n)||s[n]:t,l=e.style;return s&&(""!==u||b.contains(e.ownerDocument,e)||(u=b.style(e,n)),Yt.test(u)&&Ut.test(n)&&(i=l.width,o=l.minWidth,a=l.maxWidth,l.minWidth=l.maxWidth=l.width=u,u=s.width,l.width=i,l.minWidth=o,l.maxWidth=a)),u}):o.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s[n]:t,l=e.style;return null==u&&l&&l[n]&&(u=l[n]),Yt.test(u)&&!zt.test(n)&&(i=l.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),l.left="fontSize"===n?"1em":u,u=l.pixelLeft+"px",l.left=i,a&&(o.left=a)),""===u?"auto":u});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=b.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=b.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=b.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=b.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=b.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style[t]),Yt.test(i))return i;r=a&&(b.support.boxSizingReliable||i===e.style[t]),i=parseFloat(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function un(e){var t=o,n=Gt[e];return n||(n=ln(e,t),"none"!==n&&n||(Pt=(Pt||b("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=ln(e,t),Pt.detach()),Gt[e]=n),n}function ln(e,t){var n=b(t.createElement(e)).appendTo(t.body),r=b.css(n[0],"display");return n.remove(),r}b.each(["height","width"],function(e,n){b.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(b.css(e,"display"))?b.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,i),i):0)}}}),b.support.opacity||(b.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentStyle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":""},set:function(e,t){var n=e.style,r=e.currentStyle,i=b.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===b.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),b(function(){b.support.reliableMarginRight||(b.cssHooks.marginRight={get:function(e,n){return n?b.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!b.support.pixelPosition&&b.fn.position&&b.each(["top","left"],function(e,n){b.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?b(e).position()[n]+"px":r):t}}})}),b.expr&&b.expr.filters&&(b.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!b.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||b.css(e,"display"))},b.expr.filters.visible=function(e){return!b.expr.filters.hidden(e)}),b.each({margin:"",padding:"",border:"Width"},function(e,t){b.cssHooks[e+t]={expand:function(n){var r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(b.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;b.fn.extend({serialize:function(){return b.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=b.prop(this,"elements");return e?b.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!b(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Nt.test(e))}).map(function(e,t){var n=b(this).val();return null==n?null:b.isArray(n)?b.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),b.param=function(e,n){var r,i=[],o=function(e,t){t=b.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encodeURIComponent(t)};if(n===t&&(n=b.ajaxSettings&&b.ajaxSettings.traditional),b.isArray(e)||e.jquery&&!b.isPlainObject(e))b.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(b.isArray(t))b.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==b.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}b.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){b.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),b.fn.hover=function(e,t){return this.mouseenter(e).mouseleave(t||e)};var mn,yn,vn=b.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Nn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Cn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=b.fn.load,An={},jn={},Dn="*/".concat("*");try{yn=a.href}catch(Ln){yn=o.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(w)||[];if(b.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(u){var l;return o[u]=!0,b.each(e[u]||[],function(e,u){var c=u(n,r,i);return"string"!=typeof c||a||o[c]?a?!(l=c):t:(n.dataTypes.unshift(c),s(c),!1)}),l}return s(n.dataTypes[0])||!o["*"]&&s("*")}function Mn(e,n){var r,i,o=b.ajaxSettings.flatOptions||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&b.extend(!0,e,r),e}b.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,u=e.indexOf(" ");return u>=0&&(i=e.slice(u,e.length),e=e.slice(0,u)),b.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&b.ajax({url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?b("<div>").append(b.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},b.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){b.fn[t]=function(e){return this.on(t,e)}}),b.each(["get","post"],function(e,n){b[n]=function(e,r,i,o){return b.isFunction(r)&&(o=o||i,i=r,r=t),b.ajax({url:e,type:n,dataType:o,data:r,success:i})}}),b.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Nn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":e.String,"text html":!0,"text json":b.parseJSON,"text xml":b.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?Mn(Mn(e,b.ajaxSettings),t):Mn(b.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,u,l,c,p=b.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?b(f):b.event,h=b.Deferred(),g=b.Callbacks("once memory"),m=p.statusCode||{},y={},v={},x=0,T="canceled",N={readyState:0,getResponseHeader:function(e){var t;if(2===x){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===x?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return x||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return x||(p.mimeType=e),this},statusCode:function(e){var t;if(e)if(2>x)for(t in e)m[t]=[m[t],e[t]];else N.always(e[N.status]);return this},abort:function(e){var t=e||T;return l&&l.abort(t),k(0,t),this}};if(h.promise(N).complete=g.add,N.success=N.done,N.error=N.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=b.trim(p.dataType||"*").toLowerCase().match(w)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?80:443))==(mn[3]||("http:"===mn[1]?80:443)))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=b.param(p.data,p.traditional)),qn(An,p,n,N),2===x)return N;u=p.global,u&&0===b.active++&&b.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Cn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(b.lastModified[o]&&N.setRequestHeader("If-Modified-Since",b.lastModified[o]),b.etag[o]&&N.setRequestHeader("If-None-Match",b.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&N.setRequestHeader("Content-Type",p.contentType),N.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)N.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,N,p)===!1||2===x))return N.abort();T="abort";for(i in{success:1,error:1,complete:1})N[i](p[i]);if(l=qn(jn,p,n,N)){N.readyState=1,u&&d.trigger("ajaxSend",[N,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){N.abort("timeout")},p.timeout));try{x=1,l.send(y,k)}catch(C){if(!(2>x))throw C;k(-1,C)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,C=n;2!==x&&(x=2,s&&clearTimeout(s),l=t,a=i||"",N.readyState=e>0?4:0,r&&(w=_n(p,N,r)),e>=200&&300>e||304===e?(p.ifModified&&(T=N.getResponseHeader("Last-Modified"),T&&(b.lastModified[o]=T),T=N.getResponseHeader("etag"),T&&(b.etag[o]=T)),204===e?(c=!0,C="nocontent"):304===e?(c=!0,C="notmodified"):(c=Fn(p,w),C=c.state,y=c.data,v=c.error,c=!v)):(v=C,(e||!C)&&(C="error",0>e&&(e=0))),N.status=e,N.statusText=(n||C)+"",c?h.resolveWith(f,[y,C,N]):h.rejectWith(f,[N,C,v]),N.statusCode(m),m=t,u&&d.trigger(c?"ajaxSuccess":"ajaxError",[N,p,c?y:v]),g.fireWith(f,[N,C]),u&&(d.trigger("ajaxComplete",[N,p]),--b.active||b.event.trigger("ajaxStop")))}return N},getScript:function(e,n){return b.get(e,t,n,"script")},getJSON:function(e,t,n){return b.get(e,t,n,"json")}});function _n(e,n,r){var i,o,a,s,u=e.contents,l=e.dataTypes,c=e.responseFields;for(s in c)s in r&&(n[c[s]]=r[s]);while("*"===l[0])l.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in u)if(u[s]&&u[s].test(o)){l.unshift(s);break}if(l[0]in r)a=l[0];else{for(s in r){if(!l[0]||e.converters[s+" "+l[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==l[0]&&l.unshift(a),r[a]):t}function Fn(e,t){var n,r,i,o,a={},s=0,u=e.dataTypes.slice(),l=u[0];if(e.dataFilter&&(t=e.dataFilter(t,e.dataType)),u[1])for(i in e.converters)a[i.toLowerCase()]=e.converters[i];for(;r=u[++s];)if("*"!==r){if("*"!==l&&l!==r){if(i=a[l+" "+r]||a["* "+r],!i)for(n in a)if(o=n.split(" "),o[1]===r&&(i=a[l+" "+o[0]]||a["* "+o[0]])){i===!0?i=a[n]:a[n]!==!0&&(r=o[0],u.splice(s--,0,r));break}if(i!==!0)if(i&&e["throws"])t=i(t);else try{t=i(t)}catch(c){return{state:"parsererror",error:i?c:"No conversion from "+l+" to "+r}}}l=r}return{state:"success",data:t}}b.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return b.globalEval(e),e}}}),b.ajaxPrefilter("script",function(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),b.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=o.head||b("head")[0]||o.documentElement;return{send:function(t,i){n=o.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readyState))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var On=[],Bn=/(=)\?(?=&|$)|\?\?/;b.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=On.pop()||b.expando+"_"+vn++;return this[e]=!0,e}}),b.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,u=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return u||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=b.isFunction(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,u?n[u]=n[u].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||b.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,On.push(o)),s&&b.isFunction(a)&&a(s[0]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}b.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=b.ajaxSettings.xhr(),b.support.cors=!!Rn&&"withCredentials"in Rn,Rn=b.support.ajax=!!Rn,Rn&&b.ajaxTransport(function(n){if(!n.crossDomain||b.support.cors){var r;return{send:function(i,o){var a,s,u=n.xhr();if(n.username?u.open(n.type,n.url,n.async,n.username,n.password):u.open(n.type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)u[s]=n.xhrFields[s];n.mimeType&&u.overrideMimeType&&u.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)u.setRequestHeader(s,i[s])}catch(l){}u.send(n.hasContent&&n.data||null),r=function(e,i){var s,l,c,p;try{if(r&&(i||4===u.readyState))if(r=t,a&&(u.onreadystatechange=b.noop,$n&&delete Pn[a]),i)4!==u.readyState&&u.abort();else{p={},s=u.status,l=u.getAllResponseHeaders(),"string"==typeof u.responseText&&(p.text=u.responseText);try{c=u.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,l)},n.async?4===u.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},b(e).unload($n)),Pn[a]=r),u.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+x+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n,r,i=this.createTween(e,t),o=Yn.exec(t),a=i.cur(),s=+a||0,u=1,l=20;if(o){if(n=+o[2],r=o[3]||(b.cssNumber[e]?"":"px"),"px"!==r&&s){s=b.css(i.elem,e,!0)||n||1;do u=u||".5",s/=u,b.style(i.elem,e,s+r);while(u!==(u=i.cur()/a)&&1!==u&&--l)}i.unit=r,i.start=s,i.end=o[1]?s+(o[1]+1)*n:n}return i}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=b.now()}function Zn(e,t){b.each(t,function(t,n){var r=(Qn[t]||[]).concat(Qn["*"]),i=0,o=r.length;for(;o>i;i++)if(r[i].call(e,t,n))return})}function er(e,t,n){var r,i,o=0,a=Gn.length,s=b.Deferred().always(function(){delete u.elem}),u=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,l.startTime+l.duration-t),r=n/l.duration||0,o=1-r,a=0,u=l.tweens.length;for(;u>a;a++)l.tweens[a].run(o);return s.notifyWith(e,[l,o,n]),1>o&&u?n:(s.resolveWith(e,[l]),!1)},l=s.promise({elem:e,props:b.extend({},t),opts:b.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],createTween:function(t,n){var r=b.Tween(e,l.opts,t,n,l.opts.specialEasing[t]||l.opts.easing);return l.tweens.push(r),r},stop:function(t){var n=0,r=t?l.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)l.tweens[n].run(1);return t?s.resolveWith(e,[l,t]):s.rejectWith(e,[l,t]),this}}),c=l.props;for(tr(c,l.opts.specialEasing);a>o;o++)if(r=Gn[o].call(l,e,c,l.opts))return r;return Zn(l,c),b.isFunction(l.opts.start)&&l.opts.start.call(e,l),b.fx.timer(b.extend(u,{elem:e,anim:l,queue:l.opts.queue})),l.progress(l.opts.progress).done(l.opts.done,l.opts.complete).fail(l.opts.fail).always(l.opts.always)}function tr(e,t){var n,r,i,o,a;for(i in e)if(r=b.camelCase(i),o=t[r],n=e[i],b.isArray(n)&&(o=n[1],n=e[i]=n[0]),i!==r&&(e[r]=n,delete e[i]),a=b.cssHooks[r],a&&"expand"in a){n=a.expand(n),delete e[r];for(i in n)i in e||(e[i]=n[i],t[i]=o)}else t[r]=o}b.Animation=b.extend(er,{tweener:function(e,t){b.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,u,l,c,p,f=this,d=e.style,h={},g=[],m=e.nodeType&&nn(e);n.queue||(c=b._queueHooks(e,"fx"),null==c.unqueued&&(c.unqueued=0,p=c.empty.fire,c.empty.fire=function(){c.unqueued||p()}),c.unqueued++,f.always(function(){f.always(function(){c.unqueued--,b.queue(e,"fx").length||c.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overflow=[d.overflow,d.overflowX,d.overflowY],"inline"===b.css(e,"display")&&"none"===b.css(e,"float")&&(b.support.inlineBlockNeedsLayout&&"inline"!==un(e.nodeName)?d.zoom=1:d.display="inline-block")),n.overflow&&(d.overflow="hidden",b.support.shrinkWrapBlocks||f.always(function(){d.overflow=n.overflow[0],d.overflowX=n.overflow[1],d.overflowY=n.overflow[2]}));for(i in t)if(a=t[i],Vn.exec(a)){if(delete t[i],u=u||"toggle"===a,a===(m?"hide":"show"))continue;g.push(i)}if(o=g.length){s=b._data(e,"fxshow")||b._data(e,"fxshow",{}),"hidden"in s&&(m=s.hidden),u&&(s.hidden=!m),m?b(e).show():f.done(function(){b(e).hide()}),f.done(function(){var t;b._removeData(e,"fxshow");for(t in h)b.style(e,t,h[t])});for(i=0;o>i;i++)r=g[i],l=f.createTween(r,m?s[r]:0),h[r]=s[r]||b.style(e,r),r in s||(s[r]=l.start,m&&(l.end=l.start,l.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}b.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this.prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(b.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?b.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototype.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=b.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){b.fx.step[e.prop]?b.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[b.cssProps[e.prop]]||b.cssHooks[e.prop])?b.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scrollTop=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},b.each(["toggle","show","hide"],function(e,t){var n=b.fn[t];b.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),b.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=b.isEmptyObject(e),o=b.speed(t,n,r),a=function(){var t=er(this,b.extend({},e),o);a.finish=function(){t.stop(!0)},(i||b._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.queue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=b.timers,a=b._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].elem!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&b.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=b._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=b.timers,a=r?r.length:0;for(n.finish=!0,b.queue(this,e,[]),i&&i.cur&&i.cur.finish&&i.cur.finish.call(this),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["margin"+n]=r["padding"+n]=e;return t&&(r.opacity=r.width=e),r}b.each({slideDown:ir("show"),slideUp:ir("hide"),slideToggle:ir("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(e,t){b.fn[e]=function(e,n,r){return this.animate(t,e,n,r)}}),b.speed=function(e,t,n){var r=e&&"object"==typeof e?b.extend({},e):{complete:n||!n&&t||b.isFunction(e)&&e,duration:e,easing:n&&t||t&&!b.isFunction(t)&&t};return r.duration=b.fx.off?0:"number"==typeof r.duration?r.duration:r.duration in b.fx.speeds?b.fx.speeds[r.duration]:b.fx.speeds._default,(null==r.queue||r.queue===!0)&&(r.queue="fx"),r.old=r.complete,r.complete=function(){b.isFunction(r.old)&&r.old.call(this),r.queue&&b.dequeue(this,r.queue)},r},b.easing={linear:function(e){return e},swing:function(e){return.5-Math.cos(e*Math.PI)/2}},b.timers=[],b.fx=rr.prototype.init,b.fx.tick=function(){var e,n=b.timers,r=0;for(Xn=b.now();n.length>r;r++)e=n[r],e()||n[r]!==e||n.splice(r--,1);n.length||b.fx.stop(),Xn=t},b.fx.timer=function(e){e()&&b.timers.push(e)&&b.fx.start()},b.fx.interval=13,b.fx.start=function(){Un||(Un=setInterval(b.fx.tick,b.fx.interval))},b.fx.stop=function(){clearInterval(Un),Un=null},b.fx.speeds={slow:600,fast:200,_default:400},b.fx.step={},b.expr&&b.expr.filters&&(b.expr.filters.animated=function(e){return b.grep(b.timers,function(t){return e===t.elem}).length}),b.fn.offset=function(e){if(arguments.length)return e===t?this:this.each(function(t){b.offset.setOffset(this,e,t)});var n,r,o={top:0,left:0},a=this[0],s=a&&a.ownerDocument;if(s)return n=s.documentElement,b.contains(n,a)?(typeof a.getBoundingClientRect!==i&&(o=a.getBoundingClientRect()),r=or(s),{top:o.top+(r.pageYOffset||n.scrollTop)-(n.clientTop||0),left:o.left+(r.pageXOffset||n.scrollLeft)-(n.clientLeft||0)}):o},b.offset={setOffset:function(e,t,n){var r=b.css(e,"position");"static"===r&&(e.style.position="relative");var i=b(e),o=i.offset(),a=b.css(e,"top"),s=b.css(e,"left"),u=("absolute"===r||"fixed"===r)&&b.inArray("auto",[a,s])>-1,l={},c={},p,f;u?(c=i.position(),p=c.top,f=c.left):(p=parseFloat(a)||0,f=parseFloat(s)||0),b.isFunction(t)&&(t=t.call(e,n,o)),null!=t.top&&(l.top=t.top-o.top+p),null!=t.left&&(l.left=t.left-o.left+f),"using"in t?t.using.call(e,l):i.css(l)}},b.fn.extend({position:function(){if(this[0]){var e,t,n={top:0,left:0},r=this[0];return"fixed"===b.css(r,"position")?t=r.getBoundingClientRect():(e=this.offsetParent(),t=this.offset(),b.nodeName(e[0],"html")||(n=e.offset()),n.top+=b.css(e[0],"borderTopWidth",!0),n.left+=b.css(e[0],"borderLeftWidth",!0)),{top:t.top-n.top-b.css(r,"marginTop",!0),left:t.left-n.left-b.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent||o.documentElement;while(e&&!b.nodeName(e,"html")&&"static"===b.css(e,"position"))e=e.offsetParent;return e||o.documentElement})}}),b.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(e,n){var r=/Y/.test(n);b.fn[e]=function(i){return b.access(this,function(e,i,o){var a=or(e);return o===t?a?n in a?a[n]:a.document.documentElement[i]:e[i]:(a?a.scrollTo(r?b(a).scrollLeft():o,r?o:b(a).scrollTop()):e[i]=o,t)},e,i,arguments.length,null)}});function or(e){return b.isWindow(e)?e:9===e.nodeType?e.defaultView||e.parentWindow:!1}b.each({Height:"height",Width:"width"},function(e,n){b.each({padding:"inner"+e,content:n,"":"outer"+e},function(r,i){b.fn[i]=function(i,o){var a=arguments.length&&(r||"boolean"!=typeof i),s=r||(i===!0||o===!0?"margin":"border");return b.access(this,function(n,r,i){var o;return b.isWindow(n)?n.document.documentElement["client"+e]:9===n.nodeType?(o=n.documentElement,Math.max(n.body["scroll"+e],o["scroll"+e],n.body["offset"+e],o["offset"+e],o["client"+e])):i===t?b.css(n,r,s):b.style(n,r,i,s)},n,a?i:t,a,null)}})}),e.jQuery=e.$=b,"function"==typeof define&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return b})})(window);
diff --git a/asterix-app/src/main/resources/webui/static/js/smoothie.js b/asterix-app/src/main/resources/webui/static/js/smoothie.js
new file mode 100644
index 0000000..60c6624
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/static/js/smoothie.js
@@ -0,0 +1,558 @@
+
+;(function(exports) {
+
+  var Util = {
+    extend: function() {
+      arguments[0] = arguments[0] || {};
+      for (var i = 1; i < arguments.length; i++)
+      {
+        for (var key in arguments[i])
+        {
+          if (arguments[i].hasOwnProperty(key))
+          {
+            if (typeof(arguments[i][key]) === 'object') {
+              if (arguments[i][key] instanceof Array) {
+                arguments[0][key] = arguments[i][key];
+              } else {
+                arguments[0][key] = Util.extend(arguments[0][key], arguments[i][key]);
+              }
+            } else {
+              arguments[0][key] = arguments[i][key];
+            }
+          }
+        }
+      }
+      return arguments[0];
+    }
+  };
+
+  /**
+   * Initialises a new <code>TimeSeries</code> with optional data options.
+   *
+   * Options are of the form (defaults shown):
+   *
+   * <pre>
+   * {
+   *   resetBounds: true,        
+   *   resetBoundsInterval: 3000 
+   * }
+   * </pre>
+   *
+   * Presentation options for TimeSeries are specified as an argument to <code>SmoothieChart.addTimeSeries</code>.
+   *
+   * @constructor
+   */
+  function TimeSeries(options) {
+    this.options = Util.extend({}, TimeSeries.defaultOptions, options);
+    this.data = [];
+    this.maxValue = Number.NaN; 
+    this.minValue = Number.NaN; 
+  }
+
+  TimeSeries.defaultOptions = {
+    resetBoundsInterval: 3000,
+    resetBounds: false
+  };
+
+  /**
+   * Recalculate the min/max values for this <code>TimeSeries</code> object.
+   *
+   * This causes the graph to scale itself in the y-axis.
+   */
+  TimeSeries.prototype.resetBounds = function() {
+    if (this.data.length) {
+      
+      this.maxValue = this.data[0][1];
+      this.minValue = this.data[0][1];
+      for (var i = 1; i < this.data.length; i++) {
+        var value = this.data[i][1];
+        if (value > this.maxValue) {
+          this.maxValue = value;
+        }
+        if (value < this.minValue) {
+          this.minValue = value;
+        }
+      }
+    } else {
+      
+      this.maxValue = Number.NaN;
+      this.minValue = Number.NaN;
+    }
+  };
+
+  /**
+   * Adds a new data point to the <code>TimeSeries</code>, preserving chronological order.
+   *
+   * @param timestamp the position, in time, of this data point
+   * @param value the value of this data point
+   * @param sumRepeatedTimeStampValues if <code>timestamp</code> has an exact match in the series, this flag controls
+   * whether it is replaced, or the values summed (defaults to false.)
+   */
+  TimeSeries.prototype.append = function(timestamp, value, sumRepeatedTimeStampValues) {
+    
+    var i = this.data.length - 1;
+    while (i > 0 && this.data[i][0] > timestamp) {
+      i--;
+    }
+
+    if (this.data.length > 0 && this.data[i][0] === timestamp) {
+      
+      if (sumRepeatedTimeStampValues) {
+        
+        this.data[i][1] += value;
+        value = this.data[i][1];
+      } else {
+        
+        this.data[i][1] = value;
+      }
+    } else if (i < this.data.length - 1) {
+      
+      this.data.splice(i + 1, 0, [timestamp, value]);
+    } else {
+      
+      this.data.push([timestamp, value]);
+    }
+
+    this.maxValue = isNaN(this.maxValue) ? value : Math.max(this.maxValue, value);
+    this.minValue = isNaN(this.minValue) ? value : Math.min(this.minValue, value);
+  };
+
+  TimeSeries.prototype.dropOldData = function(oldestValidTime, maxDataSetLength) {
+    
+    
+    var removeCount = 0;
+    while (this.data.length - removeCount >= maxDataSetLength && this.data[removeCount + 1][0] < oldestValidTime) {
+      removeCount++;
+    }
+    if (removeCount !== 0) {
+      this.data.splice(0, removeCount);
+    }
+  };
+
+  /**
+   * Initialises a new <code>SmoothieChart</code>.
+   *
+   * Options are optional, and should be of the form below. Just specify the values you
+   * need and the rest will be given sensible defaults as shown:
+   *
+   * <pre>
+   * {
+   *   minValue: undefined,        
+   *   maxValue: undefined,        
+   *   maxValueScale: 1,           
+   *   yRangeFunction: undefined,  
+   *   scaleSmoothing: 0.125,      
+   *   millisPerPixel: 20,         
+   *   maxDataSetLength: 2,
+   *   interpolation: 'bezier'     
+   *   timestampFormatter: null,   
+   *   horizontalLines: [],        
+   *   grid:
+   *   {
+   *     fillStyle: '#000000',     
+   *     lineWidth: 1,             
+   *     strokeStyle: '#777777',   
+   *     millisPerLine: 1000,      
+   *     sharpLines: false,        
+   *     verticalSections: 2,      
+   *     borderVisible: true       
+   *   },
+   *   labels
+   *   {
+   *     disabled: false,          
+   *     fillStyle: '#ffffff',     
+   *     fontSize: 15,
+   *     fontFamily: 'sans-serif',
+   *     precision: 2
+   *   },
+   * }
+   * </pre>
+   *
+   * @constructor
+   */
+  function SmoothieChart(options) {
+    this.options = Util.extend({}, SmoothieChart.defaultChartOptions, options);
+    this.seriesSet = [];
+    this.currentValueRange = 1;
+    this.currentVisMinValue = 0;
+  }
+
+  SmoothieChart.defaultChartOptions = {
+    millisPerPixel: 20,
+    maxValueScale: 1,
+    interpolation: 'bezier',
+    scaleSmoothing: 0.125,
+    maxDataSetLength: 2,
+    grid: {
+      fillStyle: '#000000',
+      strokeStyle: '#777777',
+      lineWidth: 1,
+      sharpLines: false,
+      millisPerLine: 1000,
+      verticalSections: 2,
+      borderVisible: true
+    },
+    labels: {
+      fillStyle: '#ffffff',
+      disabled: false,
+      fontSize: 10,
+      fontFamily: 'monospace',
+      precision: 2
+    },
+    horizontalLines: []
+  };
+
+  SmoothieChart.AnimateCompatibility = (function() {
+    var lastTime = 0,
+        requestAnimationFrame = function(callback, element) {
+          var requestAnimationFrame =
+            window.requestAnimationFrame        ||
+            window.webkitRequestAnimationFrame  ||
+            window.mozRequestAnimationFrame     ||
+            window.oRequestAnimationFrame       ||
+            window.msRequestAnimationFrame      ||
+            function(callback) {
+              var currTime = new Date().getTime(),
+                  timeToCall = Math.max(0, 16 - (currTime - lastTime)),
+                  id = window.setTimeout(function() {
+                    callback(currTime + timeToCall);
+                  }, timeToCall);
+              lastTime = currTime + timeToCall;
+              return id;
+            };
+          return requestAnimationFrame.call(window, callback, element);
+        },
+        cancelAnimationFrame = function(id) {
+          var cancelAnimationFrame =
+            window.cancelAnimationFrame ||
+            function(id) {
+              clearTimeout(id);
+            };
+          return cancelAnimationFrame.call(window, id);
+        };
+
+    return {
+      requestAnimationFrame: requestAnimationFrame,
+      cancelAnimationFrame: cancelAnimationFrame
+    };
+  })();
+
+  SmoothieChart.defaultSeriesPresentationOptions = {
+    lineWidth: 1,
+    strokeStyle: '#ffffff'
+  };
+
+  /**
+   * Adds a <code>TimeSeries</code> to this chart, with optional presentation options.
+   *
+   * Presentation options should be of the form (defaults shown):
+   *
+   * <pre>
+   * {
+   *   lineWidth: 1,
+   *   strokeStyle: '#ffffff',
+   *   fillStyle: undefined
+   * }
+   * </pre>
+   */
+  SmoothieChart.prototype.addTimeSeries = function(timeSeries, options) {
+    this.seriesSet.push({timeSeries: timeSeries, options: Util.extend({}, SmoothieChart.defaultSeriesPresentationOptions, options)});
+    if (timeSeries.options.resetBounds && timeSeries.options.resetBoundsInterval > 0) {
+      timeSeries.resetBoundsTimerId = setInterval(
+        function() {
+          timeSeries.resetBounds();
+        },
+        timeSeries.options.resetBoundsInterval
+      );
+    }
+  };
+
+  /**
+   * Removes the specified <code>TimeSeries</code> from the chart.
+   */
+  SmoothieChart.prototype.removeTimeSeries = function(timeSeries) {
+    var numSeries = this.seriesSet.length;
+    for (var i = 0; i < numSeries; i++) {
+      if (this.seriesSet[i].timeSeries === timeSeries) {
+        this.seriesSet.splice(i, 1);
+        break;
+      }
+    }
+    if (timeSeries.resetBoundsTimerId) {
+      clearInterval(timeSeries.resetBoundsTimerId);
+    }
+  };
+
+  /**
+   * Instructs the <code>SmoothieChart</code> to start rendering to the provided canvas, with specified delay.
+   *
+   * @param canvas the target canvas element
+   * @param delayMillis an amount of time to wait before a data point is shown. This can prevent the end of the series
+   * from appearing on screen, with new values flashing into view, at the expense of some latency.
+   */
+  SmoothieChart.prototype.streamTo = function(canvas, delayMillis) {
+    this.canvas = canvas;
+    this.delay = delayMillis;
+    this.start();
+  };
+
+  /**
+   * Starts the animation of this chart.
+   */
+  SmoothieChart.prototype.start = function() {
+    if (this.frame) {
+      return;
+    }
+
+    var animate = function() {
+      this.frame = SmoothieChart.AnimateCompatibility.requestAnimationFrame(function() {
+        this.render();
+        animate();
+      }.bind(this));
+    }.bind(this);
+
+    animate();
+  };
+
+  /**
+   * Stops the animation of this chart.
+   */
+  SmoothieChart.prototype.stop = function() {
+    if (this.frame) {
+      SmoothieChart.AnimateCompatibility.cancelAnimationFrame(this.frame);
+      delete this.frame;
+    }
+  };
+
+  SmoothieChart.prototype.updateValueRange = function() {
+    var chartOptions = this.options,
+        chartMaxValue = Number.NaN,
+        chartMinValue = Number.NaN;
+
+    for (var d = 0; d < this.seriesSet.length; d++) {
+      var timeSeries = this.seriesSet[d].timeSeries;
+      if (!isNaN(timeSeries.maxValue)) {
+        chartMaxValue = !isNaN(chartMaxValue) ? Math.max(chartMaxValue, timeSeries.maxValue) : timeSeries.maxValue;
+      }
+
+      if (!isNaN(timeSeries.minValue)) {
+        chartMinValue = !isNaN(chartMinValue) ? Math.min(chartMinValue, timeSeries.minValue) : timeSeries.minValue;
+      }
+    }
+
+    if (chartOptions.maxValue != null) {
+      chartMaxValue = chartOptions.maxValue;
+    } else {
+      chartMaxValue *= chartOptions.maxValueScale;
+    }
+
+    if (chartOptions.minValue != null) {
+      chartMinValue = chartOptions.minValue;
+    }
+
+    if (this.options.yRangeFunction) {
+      var range = this.options.yRangeFunction({min: chartMinValue, max: chartMaxValue});
+      chartMinValue = range.min;
+      chartMaxValue = range.max;
+    }
+
+    if (!isNaN(chartMaxValue) && !isNaN(chartMinValue)) {
+      var targetValueRange = chartMaxValue - chartMinValue;
+      this.currentValueRange += chartOptions.scaleSmoothing * (targetValueRange - this.currentValueRange);
+      this.currentVisMinValue += chartOptions.scaleSmoothing * (chartMinValue - this.currentVisMinValue);
+    }
+
+    this.valueRange = { min: chartMinValue, max: chartMaxValue };
+  };
+
+  SmoothieChart.prototype.render = function(canvas, time) {
+    canvas = canvas || this.canvas;
+    time = time || new Date().getTime() - (this.delay || 0);
+
+
+    time -= time % this.options.millisPerPixel;
+
+    var context = canvas.getContext('2d'),
+        chartOptions = this.options,
+        dimensions = { top: 0, left: 0, width: canvas.clientWidth, height: canvas.clientHeight },
+        oldestValidTime = time - (dimensions.width * chartOptions.millisPerPixel),
+        valueToYPixel = function(value) {
+          var offset = value - this.currentVisMinValue;
+          return this.currentValueRange === 0
+            ? dimensions.height
+            : dimensions.height - (Math.round((offset / this.currentValueRange) * dimensions.height));
+        }.bind(this),
+        timeToXPixel = function(t) {
+          return Math.round(dimensions.width - ((time - t) / chartOptions.millisPerPixel));
+        };
+
+    this.updateValueRange();
+
+    context.font = chartOptions.labels.fontSize + 'px ' + chartOptions.labels.fontFamily;
+
+    context.save();
+
+    context.translate(dimensions.left, dimensions.top);
+
+    context.beginPath();
+    context.rect(0, 0, dimensions.width, dimensions.height);
+    context.clip();
+
+    context.save();
+    context.fillStyle = chartOptions.grid.fillStyle;
+    context.clearRect(0, 0, dimensions.width, dimensions.height);
+    context.fillRect(0, 0, dimensions.width, dimensions.height);
+    context.restore();
+
+    context.save();
+    context.lineWidth = chartOptions.grid.lineWidth;
+    context.strokeStyle = chartOptions.grid.strokeStyle;
+    if (chartOptions.grid.millisPerLine > 0) {
+      var textUntilX = dimensions.width - context.measureText(minValueString).width + 4;
+      for (var t = time - (time % chartOptions.grid.millisPerLine);
+           t >= oldestValidTime;
+           t -= chartOptions.grid.millisPerLine) {
+        var gx = timeToXPixel(t);
+        if (chartOptions.grid.sharpLines) {
+          gx -= 0.5;
+        }
+        context.beginPath();
+        context.moveTo(gx, 0);
+        context.lineTo(gx, dimensions.height);
+        context.stroke();
+        context.closePath();
+
+        if (chartOptions.timestampFormatter && gx < textUntilX) {
+          
+          var tx = new Date(t),
+            ts = chartOptions.timestampFormatter(tx),
+            tsWidth = context.measureText(ts).width;
+          textUntilX = gx - tsWidth - 2;
+          context.fillStyle = chartOptions.labels.fillStyle;
+          context.fillText(ts, gx - tsWidth, dimensions.height - 2);
+        }
+      }
+    }
+
+    for (var v = 1; v < chartOptions.grid.verticalSections; v++) {
+      var gy = Math.round(v * dimensions.height / chartOptions.grid.verticalSections);
+      if (chartOptions.grid.sharpLines) {
+        gy -= 0.5;
+      }
+      context.beginPath();
+      context.moveTo(0, gy);
+      context.lineTo(dimensions.width, gy);
+      context.stroke();
+      context.closePath();
+    }
+    if (chartOptions.grid.borderVisible) {
+      context.beginPath();
+      context.strokeRect(0, 0, dimensions.width, dimensions.height);
+      context.closePath();
+    }
+    context.restore();
+
+    if (chartOptions.horizontalLines && chartOptions.horizontalLines.length) {
+      for (var hl = 0; hl < chartOptions.horizontalLines.length; hl++) {
+        var line = chartOptions.horizontalLines[hl],
+            hly = Math.round(valueToYPixel(line.value)) - 0.5;
+        context.strokeStyle = line.color || '#ffffff';
+        context.lineWidth = line.lineWidth || 1;
+        context.beginPath();
+        context.moveTo(0, hly);
+        context.lineTo(dimensions.width, hly);
+        context.stroke();
+        context.closePath();
+      }
+    }
+
+    for (var d = 0; d < this.seriesSet.length; d++) {
+      context.save();
+      var timeSeries = this.seriesSet[d].timeSeries,
+          dataSet = timeSeries.data,
+          seriesOptions = this.seriesSet[d].options;
+
+      timeSeries.dropOldData(oldestValidTime, chartOptions.maxDataSetLength);
+
+      context.lineWidth = seriesOptions.lineWidth;
+      context.strokeStyle = seriesOptions.strokeStyle;
+      context.beginPath();
+      var firstX = 0, lastX = 0, lastY = 0;
+      for (var i = 0; i < dataSet.length && dataSet.length !== 1; i++) {
+        var x = timeToXPixel(dataSet[i][0]),
+            y = valueToYPixel(dataSet[i][1]);
+
+        if (i === 0) {
+          firstX = x;
+          context.moveTo(x, y);
+        } else {
+          switch (chartOptions.interpolation) {
+            case "linear":
+            case "line": {
+              context.lineTo(x,y);
+              break;
+            }
+            case "bezier":
+            default: {
+              
+              
+              
+              
+              
+              
+              
+              
+              
+              context.bezierCurveTo( 
+                Math.round((lastX + x) / 2), lastY, 
+                Math.round((lastX + x)) / 2, y, 
+                x, y); 
+              break;
+            }
+          }
+        }
+
+        lastX = x; lastY = y;
+      }
+
+      if (dataSet.length > 1) {
+        if (seriesOptions.fillStyle) {
+          
+          context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, lastY);
+          context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, dimensions.height + seriesOptions.lineWidth + 1);
+          context.lineTo(firstX, dimensions.height + seriesOptions.lineWidth);
+          context.fillStyle = seriesOptions.fillStyle;
+          context.fill();
+        }
+
+        if (seriesOptions.strokeStyle && seriesOptions.strokeStyle !== 'none') {
+          context.stroke();
+        }
+        context.closePath();
+      }
+      context.restore();
+    }
+
+    
+    if (!chartOptions.labels.disabled && !isNaN(this.valueRange.min) && !isNaN(this.valueRange.max)) {
+      var maxValueString = parseFloat(this.valueRange.max).toFixed(chartOptions.labels.precision),
+          minValueString = parseFloat(this.valueRange.min).toFixed(chartOptions.labels.precision);
+      context.fillStyle = chartOptions.labels.fillStyle;
+      context.fillText(maxValueString, dimensions.width - context.measureText(maxValueString).width - 2, chartOptions.labels.fontSize);
+      context.fillText(minValueString, dimensions.width - context.measureText(minValueString).width - 2, dimensions.height - 2);
+    }
+
+    context.restore(); 
+  };
+
+  
+  SmoothieChart.timeFormatter = function(date) {
+    function pad2(number) { return (number < 10 ? '0' : '') + number }
+    return pad2(date.getHours()) + ':' + pad2(date.getMinutes()) + ':' + pad2(date.getSeconds());
+  };
+
+  exports.TimeSeries = TimeSeries;
+  exports.SmoothieChart = SmoothieChart;
+
+})(typeof exports === 'undefined' ?  this : exports);
+
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 36bf23c..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
@@ -17,6 +17,8 @@
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
@@ -40,6 +42,9 @@
  */
 @RunWith(Parameterized.class)
 public class ExecutionTest {
+
+    private static final Logger LOGGER = Logger.getLogger(ExecutionTest.class.getName());
+
     private static final String PATH_ACTUAL = "rttest/";
     private static final String PATH_BASE = "src/test/resources/runtimets/";
 
@@ -50,6 +55,10 @@
 
     @BeforeClass
     public static void setUp() throws Exception {
+        System.out.println("Starting setup");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Starting setup");
+        }
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
         System.setProperty(GlobalConfig.WEB_SERVER_PORT_PROPERTY, "19002");
         File outdir = new File(PATH_ACTUAL);
@@ -60,16 +69,22 @@
 
         deleteTransactionLogs();
 
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("initializing pseudo cluster");
+        }
         AsterixHyracksIntegrationUtil.init();
 
-        // TODO: Uncomment when hadoop version is upgraded and adapters are
-        // ported. 
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("initializing HDFS");
+        }
+
         HDFSCluster.getInstance().setup();
 
         // Set the node resolver to be the identity resolver that expects node names 
         // to be node controller ids; a valid assumption in test environment. 
         System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
+
     }
 
     @AfterClass
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
index d6a813c..2318a04 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
@@ -88,8 +88,10 @@
     }
 
     public void cleanup() throws Exception {
-        dfsCluster.shutdown();
-        cleanupLocal();
+        if (dfsCluster != null) {
+            dfsCluster.shutdown();
+            cleanupLocal();
+        }
     }
 
     public static void main(String[] args) throws Exception {
diff --git a/asterix-app/src/test/resources/hadoop/conf/core-site.xml b/asterix-app/src/test/resources/hadoop/conf/core-site.xml
index 5b1023c..433325a 100644
--- a/asterix-app/src/test/resources/hadoop/conf/core-site.xml
+++ b/asterix-app/src/test/resources/hadoop/conf/core-site.xml
@@ -21,7 +21,7 @@
 
 <property>
     <name>fs.default.name</name>
-    <value>hdfs://127.0.0.1:31888</value>
+    <value>hdfs://192.168.0.103:31888</value>
 </property>
 <property>
     <name>hadoop.tmp.dir</name>
diff --git a/asterix-app/src/test/resources/logging.properties b/asterix-app/src/test/resources/logging.properties
index cf1457f..c904be4 100644
--- a/asterix-app/src/test/resources/logging.properties
+++ b/asterix-app/src/test/resources/logging.properties
@@ -79,3 +79,5 @@
 #edu.uci.ics.asterix.level = FINE
 #edu.uci.ics.hyracks.algebricks.level = FINE
 #edu.uci.ics.hyracks.level = INFO
+edu.uci.ics.asterix.test = INFO
+edu.uci.ics.asterix.installer.test = INFO
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 0fb70db..e29398c 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,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:01:03 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:01:03 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
+>>>>>>> master
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 9ec7ae7..e5e8615 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,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:04:09 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:04:09 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
+>>>>>>> master
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 98f9d7d..23b465c 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,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:25:30 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:25:30 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
+>>>>>>> master
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 8fb7bc2..c034bb5 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,5 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:09 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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 }
+=======
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:09 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
+>>>>>>> master
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 371ee07..64b3391 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,5 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:08 PDT 2013", "DatasetId": 103, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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 }
+=======
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:08 PDT 2013", "DatasetId": 103, "PendingOp": 0 }
+>>>>>>> master
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 4414ed0..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,6 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
+{ "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 4414ed0..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,6 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
+{ "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 b56fe7c..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,8 +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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
-{ "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": "Tue Jan 29 18:54:03 PST 2013" }
+{ "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 414d951..3788cc5 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,3 +1,17 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
@@ -6,4 +20,5 @@
 { "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+>>>>>>> master
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 2de89a6..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,60 +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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:30 PST 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": "Fri Feb 08 15:49:29 PST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:30 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:30 PST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 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 1995946..c749dcc 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,3 +1,71 @@
+<<<<<<< HEAD
+{ "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" }
+=======
 { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:04:06 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": "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": "Mon Sep 23 00:04:06 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": "Mon Sep 23 00:04:06 PDT 2013" }
@@ -64,4 +132,5 @@
 { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
+>>>>>>> master
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 607bfd1..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,11 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "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 7fdcb8d..f744e61 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,3 +1,20 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
@@ -9,4 +26,5 @@
 { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
+>>>>>>> master
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 8abc339..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,8 +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, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "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 edf8f9d..ff87550 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,3 +1,17 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
@@ -6,4 +20,5 @@
 { "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 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", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+>>>>>>> master
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 b351cfb..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,56 +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": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "string" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "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": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "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 4c22058..b2f62a2 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,3 +1,71 @@
+<<<<<<< HEAD
+{ "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" }
+=======
 { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:25:26 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": "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": "Mon Sep 23 00:25:26 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": "Mon Sep 23 00:25:26 PDT 2013" }
@@ -64,4 +132,5 @@
 { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
+>>>>>>> master
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 607bfd1..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,11 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "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 072bcde..b65906f 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,3 +1,20 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
@@ -9,4 +26,5 @@
 { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql b/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
index 1bd4b73..cd51b5e 100644
--- a/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
+++ b/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
@@ -52,4 +52,4 @@
 let $c3 := int32("320")
 let $c4 := int64("640")
 return {"int8": $c1,"int16": $c2,"int32": $c3, "int8co": $o.int8co, "int64": $c4}
-*/
\ No newline at end of file
+*/
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql
new file mode 100644
index 0000000..1883c94
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql
@@ -0,0 +1,24 @@
+/* 
+ * Test case Name  : insert-into-empty-dataset.aql
+ * Description     : Check that we can insert into an empty dataset 
+ * Expected Result : Success
+ * Date            : May 2 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineIDType as closed {
+  l_orderkey: int32, 
+  l_linenumber: int32, 
+  l_suppkey: int32
+}
+
+create dataset LineID(LineIDType)
+  primary key l_orderkey, l_linenumber;
+  
+create dataset LineID2(LineIDType)
+  primary key l_orderkey, l_linenumber;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql
new file mode 100644
index 0000000..e155837
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql
@@ -0,0 +1,35 @@
+/* 
+ * Test case Name  : insert-into-empty-dataset.aql
+ * Description     : Check that we can insert into an empty dataset 
+ * Expected Result : Success
+ * Date            : May 2 2012
+ */
+
+use dataverse test;
+
+insert into dataset LineID (
+let $x:=1
+let $y:=2
+let $z:=3
+return {
+	"l_orderkey": $x,
+	"l_linenumber": $y,
+	"l_suppkey": $z
+}
+);
+
+insert into dataset LineID (
+let $x:=2
+let $y:=3
+let $z:=4
+return {
+	"l_orderkey": $x,
+	"l_linenumber": $y,
+	"l_suppkey": $z
+}
+);
+
+insert into dataset LineID2 (
+  for $x in dataset LineID
+  return flow-record($x)
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql
new file mode 100644
index 0000000..e4763e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : insert-into-empty-dataset.aql
+ * Description     : Check that we can insert into an empty dataset 
+ * Expected Result : Success
+ * Date            : May 2 2012
+ */
+
+use dataverse test;
+
+for $c in dataset('LineID2')
+order by $c.l_orderkey, $c.l_linenumber
+return $c
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 aafd2c9..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
@@ -9,17 +9,18 @@
 create dataverse feeds;
 use dataverse feeds;
 
+
 create type TweetType as closed {
   id: string,
   username : string,
   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 01b0925..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
@@ -6,5 +6,7 @@
  */
 
 use dataverse feeds;
-  
-begin feed TweetFeed;
+
+set wait-for-completion-feed "true";
+
+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 060576e..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
@@ -8,4 +8,6 @@
   
 use dataverse feeds;
 
-begin feed TweetFeed;
+set wait-for-completion-feed "true";
+
+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
new file mode 100644
index 0000000..ec92972
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
@@ -0,0 +1,39 @@
+/*
+ * 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). 
+                  Verify the existence of data after the feed finishes.
+                  
+ * Expected Res : Success
+ * Date         : 20th Jun 2013
+ */
+
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+create type TwitterUserType as closed {
+	screen-name: string,
+	lang: string,
+	friends_count: int32,
+	statuses_count: int32,
+	name: string,
+	followers_count: int32
+} 
+
+create type TweetMessageType as closed {
+	tweetid: string,
+        user: TwitterUserType,
+        sender-location: point,
+	send-time: datetime,
+        referred-topics: {{ string }},
+	message-text: string
+}
+
+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
new file mode 100644
index 0000000..d9e5404
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
@@ -0,0 +1,15 @@
+/*
+ * 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).
+                  Verify the existence of data after the feed finishes.
+
+ * Expected Res : Success
+ * Date         : 20th Jun 2013
+ */
+  
+use dataverse feeds;
+
+set wait-for-completion-feed "true";
+
+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
new file mode 100644
index 0000000..dd520da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
@@ -0,0 +1,21 @@
+/*
+ * 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).
+                  Verify the existence of data after the feed finishes.
+
+ * Expected Res : Success
+ * Date         : 20th Jun 2013
+ */
+
+use dataverse feeds;
+
+let $totalTweets:=count(
+for $x in dataset('SyntheticTweets')
+return $x)
+return 
+(if($totalTweets > 0)
+  then 1
+else
+  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 62ac61f..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
@@ -8,6 +8,7 @@
 create dataverse feeds;
 use dataverse feeds;
 
+
 create type TweetType as closed {
   id: string,
   username : string,
@@ -16,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 0e22d6e..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
@@ -6,5 +6,7 @@
  */
 
 use dataverse feeds;
-  
-begin feed feeds.TweetFeed;
+
+set wait-for-completion-feed "true";
+ 
+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 7851440..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
@@ -8,4 +8,6 @@
 
 use dataverse feeds;
 
-begin feed feeds.TweetFeed;
+set wait-for-completion-feed "true";
+
+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/queries/open-closed/query-issue442/query-issue442.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
+++ /dev/null
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 89f66a8..2df9621 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,11 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 662, "PendingOp": 0 }
 { "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 661, "PendingOp": 0 }
 { "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 663, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 664, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 664, "PendingOp": 0 }
+>>>>>>> master
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 f9981d1..09e8d1e 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,11 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 674, "PendingOp": 0 }
 { "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 673, "PendingOp": 0 }
 { "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 675, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
+>>>>>>> master
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 2d04f8b..d776957 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,17 @@
+<<<<<<< HEAD
+{ "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 }
+=======
 { "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": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 684, "PendingOp": 0 }
 { "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 678, "PendingOp": 0 }
 { "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 681, "PendingOp": 0 }
 { "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 682, "PendingOp": 0 }
 { "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 679, "PendingOp": 0 }
 { "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 680, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 683, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 683, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm
new file mode 100644
index 0000000..3ea2b0c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm
@@ -0,0 +1,2 @@
+{ "l_orderkey": 1, "l_linenumber": 2, "l_suppkey": 3 }
+{ "l_orderkey": 2, "l_linenumber": 3, "l_suppkey": 4 }
\ No newline at end of file
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 dec4c6c..824e8c7 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,5 @@
-{ "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, "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 705, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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" }
+=======
+{ "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, "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 705, "PendingOp": 0 }
+>>>>>>> master
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 3fa1d10..2844787 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,5 @@
-{ "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", "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 707, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "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" }
+=======
+{ "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", "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 707, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm
@@ -0,0 +1 @@
+1
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 c084678..475962d 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,3 +1,9 @@
+{ "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 }
 { "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:07 PDT 2013", "DatasetId": 375, "PendingOp": 0 }
 { "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:26:07 PDT 2013", "DatasetId": 376, "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sun Sep 22 22:24:38 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 3abacd5..7c614c0 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -4350,6 +4350,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
+      <compilation-unit name="feeds_05">
+        <output-dir compare="Text">feeds_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
       <compilation-unit name="issue_230_feeds">
         <output-dir compare="Text">issue_230_feeds</output-dir>
       </compilation-unit>
diff --git a/asterix-aql/pom.xml b/asterix-aql/pom.xml
index 465c3b9..4b83e1c 100644
--- a/asterix-aql/pom.xml
+++ b/asterix-aql/pom.xml
@@ -115,6 +115,12 @@
 	</build>
 
 	<dependencies>
+                <dependency>
+                        <groupId>junit</groupId>
+                        <artifactId>junit</artifactId>
+                        <version>4.8.1</version>
+                        <scope>test</scope>
+                </dependency>
 		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
 			<artifactId>asterix-common</artifactId>
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 34693e4..ca66a97 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,
         COMPACT
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 a119e2a..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
+++ /dev/null
@@ -1,123 +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.FunctionConstants;
-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;
-
-public class BeginFeedStatement implements Statement {
-
-    private final Identifier dataverseName;
-    private final Identifier datasetName;
-    private Query query;
-    private int varCounter;
-
-    public BeginFeedStatement(Identifier dataverseName, Identifier datasetName, int varCounter) {
-        this.dataverseName = dataverseName;
-        this.datasetName = datasetName;
-        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:=" + 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;
-    }
-
-    @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);
-    }
-
-}
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..ec8b6be
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java
@@ -0,0 +1,184 @@
+/*
+ * 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.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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 adapterOutputType = null;
+        if (appliedFunction != null) {
+            function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
+            if (function == null) {
+                throw new MetadataException(" Unknown function " + function);
+            } else if (function.getParams().size() > 1) {
+                throw new MetadataException(" Incompatible function: " + appliedFunction
+                        + " Number if arguments must be 1");
+            }
+        }
+
+        org.apache.commons.lang3.tuple.Pair<IAdapterFactory, ARecordType> factoryOutput = null;
+        try {
+            factoryOutput = FeedUtil.getFeedFactoryAndOutput(sourceFeed, mdTxnCtx);
+            adapterOutputType = factoryOutput.getRight().getTypeName();
+        } catch (AlgebricksException ae) {
+            throw new MetadataException(ae);
+        }
+
+        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 + "'" + "," + "'" + adapterOutputType + "'"
+                    + "," + "'" + 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 + "'" + "," + "'"
+                        + adapterOutputType + "'" + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+                builder.append(" let $y:=(" + function.getFunctionBody() + ")" + " return $y");
+            } else {
+                builder.append(" (" + " for $x in feed-ingest ('" + feedName + "'" + "," + "'" + adapterOutputType
+                        + "'" + "," + "'" + 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 aac2b6b..22b63fa 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,11 +21,12 @@
 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.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -82,462 +84,517 @@
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 
 public class AQLPrintVisitor implements IAqlVisitorWithVoidReturn<Integer> {
-    // private int level =0;
-    private final PrintWriter out;
+	// private int level =0;
+	private final PrintWriter out;
 
-    public AQLPrintVisitor() {
-        out = new PrintWriter(System.out);
-    }
+	public AQLPrintVisitor() {
+		out = new PrintWriter(System.out);
+	}
 
-    public AQLPrintVisitor(PrintWriter out) {
-        this.out = out;
-    }
+	public AQLPrintVisitor(PrintWriter out) {
+		this.out = out;
+	}
 
-    private String skip(int step) {
-        StringBuffer sb = new StringBuffer();
-        for (int i = 0; i < step; i++)
-            sb.append("  ");
-        return sb.toString();
-    }
+	private String skip(int step) {
+		StringBuffer sb = new StringBuffer();
+		for (int i = 0; i < step; i++)
+			sb.append("  ");
+		return sb.toString();
+	}
 
-    @Override
-    public void visit(Query q, Integer step) throws AsterixException {
-        if (q.getBody() != null) {
-            out.println("Query:");
-            q.getBody().accept(this, step);
-        } else {
-            out.println("No query.");
-        }
-    }
+	@Override
+	public void visit(Query q, Integer step) throws AsterixException {
+		if (q.getBody() != null) {
+			out.println("Query:");
+			q.getBody().accept(this, step);
+		} else {
+			out.println("No query.");
+		}
+	}
 
-    @Override
-    public void visit(LiteralExpr l, Integer step) {
-        Literal lc = l.getValue();
-        if (lc.getLiteralType().equals(Literal.Type.TRUE) || lc.getLiteralType().equals(Literal.Type.FALSE)
-                || lc.getLiteralType().equals(Literal.Type.NULL)) {
-            out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "]");
-        } else {
-            out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "] ["
-                    + l.getValue().getStringValue() + "] ");
-        }
-    }
+	@Override
+	public void visit(LiteralExpr l, Integer step) {
+		Literal lc = l.getValue();
+		if (lc.getLiteralType().equals(Literal.Type.TRUE)
+				|| lc.getLiteralType().equals(Literal.Type.FALSE)
+				|| lc.getLiteralType().equals(Literal.Type.NULL)) {
+			out.println(skip(step) + "LiteralExpr ["
+					+ l.getValue().getLiteralType() + "]");
+		} else {
+			out.println(skip(step) + "LiteralExpr ["
+					+ l.getValue().getLiteralType() + "] ["
+					+ l.getValue().getStringValue() + "] ");
+		}
+	}
 
-    @Override
-    public void visit(VariableExpr v, Integer step) {
-        out.println(skip(step) + "Variable [ Name=" + v.getVar().getValue() + " Id=" + v.getVar().getId() + " ]");
-    }
+	@Override
+	public void visit(VariableExpr v, Integer step) {
+		out.println(skip(step) + "Variable [ Name=" + v.getVar().getValue()
+				+ " Id=" + v.getVar().getId() + " ]");
+	}
 
-    @Override
-    public void visit(ListConstructor lc, Integer step) throws AsterixException {
-        boolean ordered = false;
-        if (lc.getType().equals(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)) {
-            ordered = true;
-        }
+	@Override
+	public void visit(ListConstructor lc, Integer step) throws AsterixException {
+		boolean ordered = false;
+		if (lc.getType().equals(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)) {
+			ordered = true;
+		}
 
-        out.println(skip(step) + (ordered == true ? "OrderedListConstructor " : "UnorderedListConstructor ") + "[");
-        for (Expression e : lc.getExprList()) {
-            e.accept(this, step + 1);
-        }
-        out.println(skip(step) + "]");
-    }
+		out.println(skip(step)
+				+ (ordered == true ? "OrderedListConstructor "
+						: "UnorderedListConstructor ") + "[");
+		for (Expression e : lc.getExprList()) {
+			e.accept(this, step + 1);
+		}
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(RecordConstructor rc, Integer step) throws AsterixException {
-        out.println(skip(step) + "RecordConstructor [");
-        // fbList accept visitor
-        for (FieldBinding fb : rc.getFbList()) {
-            out.println(skip(step + 1) + "(");
-            fb.getLeftExpr().accept(this, step + 2);
-            out.println(skip(step + 2) + ":");
-            fb.getRightExpr().accept(this, step + 2);
-            out.println(skip(step + 1) + ")");
-        }
-        out.println(skip(step) + "]");
-    }
+	@Override
+	public void visit(RecordConstructor rc, Integer step)
+			throws AsterixException {
+		out.println(skip(step) + "RecordConstructor [");
+		// fbList accept visitor
+		for (FieldBinding fb : rc.getFbList()) {
+			out.println(skip(step + 1) + "(");
+			fb.getLeftExpr().accept(this, step + 2);
+			out.println(skip(step + 2) + ":");
+			fb.getRightExpr().accept(this, step + 2);
+			out.println(skip(step + 1) + ")");
+		}
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(CallExpr pf, Integer step) throws AsterixException {
-        out.println(skip(step) + "FunctionCall " + pf.getFunctionSignature().toString() + "[");
-        for (Expression expr : pf.getExprList()) {
-            expr.accept(this, step + 1);
-        }
-        out.println(skip(step) + "]");
-    }
+	@Override
+	public void visit(CallExpr pf, Integer step) throws AsterixException {
+		out.println(skip(step) + "FunctionCall "
+				+ pf.getFunctionSignature().toString() + "[");
+		for (Expression expr : pf.getExprList()) {
+			expr.accept(this, step + 1);
+		}
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(OperatorExpr ifbo, Integer step) throws AsterixException {
-        List<Expression> exprList = ifbo.getExprList();
-        List<OperatorType> opList = ifbo.getOpList();
-        if (ifbo.isCurrentop()) {
-            out.println(skip(step) + "OperatorExpr [");
-            exprList.get(0).accept(this, step + 1);
-            for (int i = 1; i < exprList.size(); i++) {
-                out.println(skip(step + 1) + opList.get(i - 1));
-                exprList.get(i).accept(this, step + 1);
-            }
-            out.println(skip(step) + "]");
-        } else {
-            exprList.get(0).accept(this, step);
-        }
+	@Override
+	public void visit(OperatorExpr ifbo, Integer step) throws AsterixException {
+		List<Expression> exprList = ifbo.getExprList();
+		List<OperatorType> opList = ifbo.getOpList();
+		if (ifbo.isCurrentop()) {
+			out.println(skip(step) + "OperatorExpr [");
+			exprList.get(0).accept(this, step + 1);
+			for (int i = 1; i < exprList.size(); i++) {
+				out.println(skip(step + 1) + opList.get(i - 1));
+				exprList.get(i).accept(this, step + 1);
+			}
+			out.println(skip(step) + "]");
+		} else {
+			exprList.get(0).accept(this, step);
+		}
 
-    }
+	}
 
-    @Override
-    public void visit(IfExpr ifexpr, Integer step) throws AsterixException {
-        out.println(skip(step) + "IfExpr [");
-        out.println(skip(step + 1) + "Condition:");
-        ifexpr.getCondExpr().accept(this, step + 2);
-        out.println(skip(step + 1) + "Then:");
-        ifexpr.getThenExpr().accept(this, step + 2);
-        out.println(skip(step + 1) + "Else:");
-        ifexpr.getElseExpr().accept(this, step + 2);
-        out.println(skip(step) + "]");
-    }
+	@Override
+	public void visit(IfExpr ifexpr, Integer step) throws AsterixException {
+		out.println(skip(step) + "IfExpr [");
+		out.println(skip(step + 1) + "Condition:");
+		ifexpr.getCondExpr().accept(this, step + 2);
+		out.println(skip(step + 1) + "Then:");
+		ifexpr.getThenExpr().accept(this, step + 2);
+		out.println(skip(step + 1) + "Else:");
+		ifexpr.getElseExpr().accept(this, step + 2);
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(FLWOGRExpression flwor, Integer step) throws AsterixException {
-        out.println(skip(step) + "FLWOGR [");
-        for (Clause cl : flwor.getClauseList()) {
-            cl.accept(this, step + 1);
-        }
-        out.println(skip(step + 1) + "Return");
-        flwor.getReturnExpr().accept(this, step + 2);
-        out.println(skip(step) + "]");
-    }
+	@Override
+	public void visit(FLWOGRExpression flwor, Integer step)
+			throws AsterixException {
+		out.println(skip(step) + "FLWOGR [");
+		for (Clause cl : flwor.getClauseList()) {
+			cl.accept(this, step + 1);
+		}
+		out.println(skip(step + 1) + "Return");
+		flwor.getReturnExpr().accept(this, step + 2);
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(QuantifiedExpression qe, Integer step) throws AsterixException {
-        out.println(skip(step) + "QuantifiedExpression " + qe.getQuantifier() + " [");
-        // quantifiedList accept visitor
-        for (QuantifiedPair pair : qe.getQuantifiedList()) {
-            out.print(skip(step + 1) + "[");
-            pair.getVarExpr().accept(this, 0);
-            out.println(skip(step + 1) + "In");
-            pair.getExpr().accept(this, step + 2);
-            out.println(skip(step + 1) + "]");
-        }
-        out.println(skip(step + 1) + "Satifies [");
-        qe.getSatisfiesExpr().accept(this, step + 2);
-        out.println(skip(step + 1) + "]");// for satifies
-        out.println(skip(step) + "]");// for quantifiedExpr
-    }
+	@Override
+	public void visit(QuantifiedExpression qe, Integer step)
+			throws AsterixException {
+		out.println(skip(step) + "QuantifiedExpression " + qe.getQuantifier()
+				+ " [");
+		// quantifiedList accept visitor
+		for (QuantifiedPair pair : qe.getQuantifiedList()) {
+			out.print(skip(step + 1) + "[");
+			pair.getVarExpr().accept(this, 0);
+			out.println(skip(step + 1) + "In");
+			pair.getExpr().accept(this, step + 2);
+			out.println(skip(step + 1) + "]");
+		}
+		out.println(skip(step + 1) + "Satifies [");
+		qe.getSatisfiesExpr().accept(this, step + 2);
+		out.println(skip(step + 1) + "]");// for satifies
+		out.println(skip(step) + "]");// for quantifiedExpr
+	}
 
-    @Override
-    public void visit(ForClause fc, Integer step) throws AsterixException {
-        out.print(skip(step) + "For ");
-        fc.getVarExpr().accept(this, 0);
-        out.println(skip(step + 1) + "In ");
-        fc.getInExpr().accept(this, step + 1);
-    }
+	@Override
+	public void visit(ForClause fc, Integer step) throws AsterixException {
+		out.print(skip(step) + "For ");
+		fc.getVarExpr().accept(this, 0);
+		out.println(skip(step + 1) + "In ");
+		fc.getInExpr().accept(this, step + 1);
+	}
 
-    @Override
-    public void visit(LetClause lc, Integer step) throws AsterixException {
-        out.print(skip(step) + "Let ");
-        lc.getVarExpr().accept(this, 0);
-        out.println(skip(step + 1) + ":= ");
-        lc.getBindingExpr().accept(this, step + 1);
-    }
+	@Override
+	public void visit(LetClause lc, Integer step) throws AsterixException {
+		out.print(skip(step) + "Let ");
+		lc.getVarExpr().accept(this, 0);
+		out.println(skip(step + 1) + ":= ");
+		lc.getBindingExpr().accept(this, step + 1);
+	}
 
-    @Override
-    public void visit(WhereClause wc, Integer step) throws AsterixException {
-        out.println(skip(step) + "Where ");
-        wc.getWhereExpr().accept(this, step + 1);
-    }
+	@Override
+	public void visit(WhereClause wc, Integer step) throws AsterixException {
+		out.println(skip(step) + "Where ");
+		wc.getWhereExpr().accept(this, step + 1);
+	}
 
-    @Override
-    public void visit(OrderbyClause oc, Integer step) throws AsterixException {
-        out.println(skip(step) + "Orderby");
-        List<OrderModifier> mlist = oc.getModifierList();
-        List<Expression> list = oc.getOrderbyList();
-        for (int i = 0; i < list.size(); i++) {
-            list.get(i).accept(this, step + 1);
-            out.println(skip(step + 1) + mlist.get(i).toString());
-        }
-        out.println(skip(step));
-    }
+	@Override
+	public void visit(OrderbyClause oc, Integer step) throws AsterixException {
+		out.println(skip(step) + "Orderby");
+		List<OrderModifier> mlist = oc.getModifierList();
+		List<Expression> list = oc.getOrderbyList();
+		for (int i = 0; i < list.size(); i++) {
+			list.get(i).accept(this, step + 1);
+			out.println(skip(step + 1) + mlist.get(i).toString());
+		}
+		out.println(skip(step));
+	}
 
-    @Override
-    public void visit(GroupbyClause gc, Integer step) throws AsterixException {
-        out.println(skip(step) + "Groupby");
-        for (GbyVariableExpressionPair pair : gc.getGbyPairList()) {
+	@Override
+	public void visit(GroupbyClause gc, Integer step) throws AsterixException {
+		out.println(skip(step) + "Groupby");
+		for (GbyVariableExpressionPair pair : gc.getGbyPairList()) {
 
-            if (pair.getVar() != null) {
-                pair.getVar().accept(this, step + 1);
-                out.println(skip(step + 1) + ":=");
-            }
+			if (pair.getVar() != null) {
+				pair.getVar().accept(this, step + 1);
+				out.println(skip(step + 1) + ":=");
+			}
 
-            pair.getExpr().accept(this, step + 1);
-        }
-        if (!gc.getDecorPairList().isEmpty()) {
-            out.println(skip(step + 1) + "Decor");
-            for (GbyVariableExpressionPair pair : gc.getDecorPairList()) {
-                if (pair.getVar() != null) {
-                    pair.getVar().accept(this, step + 1);
-                    out.println(skip(step + 1) + ":=");
-                }
-                pair.getExpr().accept(this, step + 1);
-            }
-        }
-        out.println(skip(step + 1) + "With");
-        for (VariableExpr exp : gc.getWithVarList()) {
-            exp.accept(this, step + 1);
-        }
-        out.println(skip(step));
-    }
+			pair.getExpr().accept(this, step + 1);
+		}
+		if (!gc.getDecorPairList().isEmpty()) {
+			out.println(skip(step + 1) + "Decor");
+			for (GbyVariableExpressionPair pair : gc.getDecorPairList()) {
+				if (pair.getVar() != null) {
+					pair.getVar().accept(this, step + 1);
+					out.println(skip(step + 1) + ":=");
+				}
+				pair.getExpr().accept(this, step + 1);
+			}
+		}
+		out.println(skip(step + 1) + "With");
+		for (VariableExpr exp : gc.getWithVarList()) {
+			exp.accept(this, step + 1);
+		}
+		out.println(skip(step));
+	}
 
-    @Override
-    public void visit(LimitClause lc, Integer step) throws AsterixException {
-        out.println(skip(step) + "Limit");
-        lc.getLimitExpr().accept(this, step + 1);
-        if (lc.getOffset() != null) {
-            out.println(skip(step + 1) + "Offset");
-            lc.getOffset().accept(this, step + 2);
-        }
-    }
+	@Override
+	public void visit(LimitClause lc, Integer step) throws AsterixException {
+		out.println(skip(step) + "Limit");
+		lc.getLimitExpr().accept(this, step + 1);
+		if (lc.getOffset() != null) {
+			out.println(skip(step + 1) + "Offset");
+			lc.getOffset().accept(this, step + 2);
+		}
+	}
+
+	@Override
+	public void visit(FunctionDecl fd, Integer step) throws AsterixException {
+		out.println(skip(step) + "FunctionDecl " + fd.getSignature().getName()
+				+ "(" + fd.getParamList().toString() + ") {");
+		fd.getFuncBody().accept(this, step + 1);
+		out.println(skip(step) + "}");
+		out.println();
+	}
+
+	@Override
+	public void visit(UnaryExpr u, Integer step) throws AsterixException {
+		if (u.getSign() != null) {
+			out.print(skip(step) + u.getSign() + " ");
+			u.getExpr().accept(this, 0);
+		} else
+			u.getExpr().accept(this, step);
+	}
+
+	@Override
+	public void visit(FieldAccessor fa, Integer step) throws AsterixException {
+		out.println(skip(step) + "FieldAccessor [");
+		fa.getExpr().accept(this, step + 1);
+		out.println(skip(step + 1) + "Field="
+				+ ((FieldAccessor) fa).getIdent().getValue());
+		out.println(skip(step) + "]");
+
+	}
+
+	@Override
+	public void visit(IndexAccessor fa, Integer step) throws AsterixException {
+		out.println(skip(step) + "IndexAccessor [");
+		fa.getExpr().accept(this, step + 1);
+		out.print(skip(step + 1) + "Index: ");
+		out.println((((IndexAccessor) fa).isAny() ? "ANY"
+				: ((IndexAccessor) fa).getIndex()));
 
-    @Override
-    public void visit(FunctionDecl fd, Integer step) throws AsterixException {
-        out.println(skip(step) + "FunctionDecl " + fd.getSignature().getName() + "(" + fd.getParamList().toString()
-                + ") {");
-        fd.getFuncBody().accept(this, step + 1);
-        out.println(skip(step) + "}");
-        out.println();
-    }
+		out.println(skip(step) + "]");
 
-    @Override
-    public void visit(UnaryExpr u, Integer step) throws AsterixException {
-        if (u.getSign() != null) {
-            out.print(skip(step) + u.getSign() + " ");
-            u.getExpr().accept(this, 0);
-        } else
-            u.getExpr().accept(this, step);
-    }
+	}
 
-    @Override
-    public void visit(FieldAccessor fa, Integer step) throws AsterixException {
-        out.println(skip(step) + "FieldAccessor [");
-        fa.getExpr().accept(this, step + 1);
-        out.println(skip(step + 1) + "Field=" + ((FieldAccessor) fa).getIdent().getValue());
-        out.println(skip(step) + "]");
+	@Override
+	public void visit(TypeDecl t, Integer step) throws AsterixException {
+		out.println(skip(step) + "TypeDecl " + t.getIdent() + " [");
+		t.getTypeDef().accept(this, step + 1);
+		out.println(skip(step) + "]");
+	}
 
-    }
+	@Override
+	public void visit(TypeReferenceExpression t, Integer arg)
+			throws AsterixException {
+		out.print(t.getIdent());
+	}
 
-    @Override
-    public void visit(IndexAccessor fa, Integer step) throws AsterixException {
-        out.println(skip(step) + "IndexAccessor [");
-        fa.getExpr().accept(this, step + 1);
-        out.print(skip(step + 1) + "Index: ");
-        out.println((((IndexAccessor) fa).isAny() ? "ANY" : ((IndexAccessor) fa).getIndex()));
+	@Override
+	public void visit(RecordTypeDefinition r, Integer step)
+			throws AsterixException {
+		if (r.getRecordKind() == RecordKind.CLOSED) {
+			out.print(skip(step) + "closed ");
+		} else {
+			out.print(skip(step) + "open ");
+		}
+		out.println("RecordType {");
+		Iterator<String> nameIter = r.getFieldNames().iterator();
+		Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
+		Iterator<Boolean> isnullableIter = r.getNullableFields().iterator();
+		boolean first = true;
+		while (nameIter.hasNext()) {
+			if (first) {
+				first = false;
+			} else {
+				out.println(",");
+			}
+			String name = nameIter.next();
+			TypeExpression texp = typeIter.next();
+			Boolean isNullable = isnullableIter.next();
+			out.print(skip(step + 1) + name + " : ");
+			texp.accept(this, step + 2);
+			if (isNullable) {
+				out.print("?");
+			}
+		}
+		out.println();
+		out.println(skip(step) + "}");
+	}
 
-        out.println(skip(step) + "]");
+	@Override
+	public void visit(OrderedListTypeDefinition x, Integer step)
+			throws AsterixException {
+		out.print("OrderedList [");
+		x.getItemTypeExpression().accept(this, step + 2);
+		out.println("]");
+	}
 
-    }
+	@Override
+	public void visit(UnorderedListTypeDefinition x, Integer step)
+			throws AsterixException {
+		out.print("UnorderedList <");
+		x.getItemTypeExpression().accept(this, step + 2);
+		out.println(">");
+	}
 
-    @Override
-    public void visit(TypeDecl t, Integer step) throws AsterixException {
-        out.println(skip(step) + "TypeDecl " + t.getIdent() + " [");
-        t.getTypeDef().accept(this, step + 1);
-        out.println(skip(step) + "]");
-    }
+	@Override
+	public void visit(DatasetDecl dd, Integer step) throws AsterixException {
+		if (dd.getDatasetType() == DatasetType.INTERNAL) {
+			out.println(skip(step)
+					+ "DatasetDecl"
+					+ dd.getName()
+					+ "("
+					+ dd.getItemTypeName()
+					+ ")"
+					+ " partitioned by "
+					+ ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+							.getPartitioningExprs());
+		} else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
+			out.println(skip(step) + "DatasetDecl" + dd.getName() + "("
+					+ dd.getItemTypeName() + ")" + "is an external dataset");
+		}
+	}
 
-    @Override
-    public void visit(TypeReferenceExpression t, Integer arg) throws AsterixException {
-        out.print(t.getIdent());
-    }
+	@Override
+	public void visit(DataverseDecl dv, Integer step) throws AsterixException {
+		out.println(skip(step) + "DataverseUse " + dv.getDataverseName());
+	}
 
-    @Override
-    public void visit(RecordTypeDefinition r, Integer step) throws AsterixException {
-        if (r.getRecordKind() == RecordKind.CLOSED) {
-            out.print(skip(step) + "closed ");
-        } else {
-            out.print(skip(step) + "open ");
-        }
-        out.println("RecordType {");
-        Iterator<String> nameIter = r.getFieldNames().iterator();
-        Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
-        Iterator<Boolean> isnullableIter = r.getNullableFields().iterator();
-        boolean first = true;
-        while (nameIter.hasNext()) {
-            if (first) {
-                first = false;
-            } else {
-                out.println(",");
-            }
-            String name = nameIter.next();
-            TypeExpression texp = typeIter.next();
-            Boolean isNullable = isnullableIter.next();
-            out.print(skip(step + 1) + name + " : ");
-            texp.accept(this, step + 2);
-            if (isNullable) {
-                out.print("?");
-            }
-        }
-        out.println();
-        out.println(skip(step) + "}");
-    }
+	@Override
+	public void visit(NodegroupDecl ngd, Integer step) throws AsterixException {
+		out.println(skip(step) + "Nodegroup " + ngd.getNodeControllerNames());
+	}
 
-    @Override
-    public void visit(OrderedListTypeDefinition x, Integer step) throws AsterixException {
-        out.print("OrderedList [");
-        x.getItemTypeExpression().accept(this, step + 2);
-        out.println("]");
-    }
+	@Override
+	public void visit(LoadFromFileStatement stmtLoad, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(UnorderedListTypeDefinition x, Integer step) throws AsterixException {
-        out.print("UnorderedList <");
-        x.getItemTypeExpression().accept(this, step + 2);
-        out.println(">");
-    }
+	}
 
-    @Override
-    public void visit(DatasetDecl dd, Integer step) throws AsterixException {
-        if (dd.getDatasetType() == DatasetType.INTERNAL) {
-            out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
-                    + " partitioned by " + ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs());
-        } 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");
-        }
-    }
+	@Override
+	public void visit(DropStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(DataverseDecl dv, Integer step) throws AsterixException {
-        out.println(skip(step) + "DataverseUse " + dv.getDataverseName());
-    }
+	}
 
-    @Override
-    public void visit(NodegroupDecl ngd, Integer step) throws AsterixException {
-        out.println(skip(step) + "Nodegroup " + ngd.getNodeControllerNames());
-    }
+	@Override
+	public void visit(WriteStatement ws, Integer step) throws AsterixException {
+		out.print(skip(step) + "WriteOutputTo " + ws.getNcName() + ":"
+				+ ws.getFileName());
+		if (ws.getWriterClassName() != null) {
+			out.print(" using " + ws.getWriterClassName());
+		}
+		out.println();
+	}
 
-    @Override
-    public void visit(LoadFromFileStatement stmtLoad, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	@Override
+	public void visit(SetStatement ss, Integer step) throws AsterixException {
+		out.println(skip(step) + "Set " + ss.getPropName() + "="
+				+ ss.getPropValue());
+	}
 
-    }
+	@Override
+	public void visit(CreateIndexStatement cis, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(DropStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(DisconnectFeedStatement ss, Integer step)
+			throws AsterixException {
+		out.println(skip(step) + skip(step) + ss.getFeedName() + skip(step)
+				+ ss.getDatasetName());
+	}
 
-    @Override
-    public void visit(WriteStatement ws, Integer step) throws AsterixException {
-        out.print(skip(step) + "WriteOutputTo " + ws.getNcName() + ":" + ws.getFileName());
-        if (ws.getWriterClassName() != null) {
-            out.print(" using " + ws.getWriterClassName());
-        }
-        out.println();
-    }
+	@Override
+	public void visit(UnionExpr u, Integer step) throws AsterixException {
+		out.println(skip(step) + "Union [");
+		for (Expression expr : u.getExprs()) {
+			expr.accept(this, step + 1);
+		}
+		out.println(skip(step) + "]");
+	}
 
-    @Override
-    public void visit(SetStatement ss, Integer step) throws AsterixException {
-        out.println(skip(step) + "Set " + ss.getPropName() + "=" + ss.getPropValue());
-    }
+	@Override
+	public void visit(DistinctClause dc, Integer step) throws AsterixException {
+		out.print(skip(step) + "Distinct ");
+		for (Expression expr : dc.getDistinctByExpr())
+			expr.accept(this, step + 1);
+	}
 
-    @Override
-    public void visit(CreateIndexStatement cis, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	@Override
+	public void visit(InsertStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    }
+	}
 
-    @Override
-    public void visit(ControlFeedStatement ss, Integer step) throws AsterixException {
-        out.println(skip(step) + ss.getOperationType() + skip(step) + ss.getDatasetName());
-    }
+	@Override
+	public void visit(DeleteStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(UnionExpr u, Integer step) throws AsterixException {
-        out.println(skip(step) + "Union [");
-        for (Expression expr : u.getExprs()) {
-            expr.accept(this, step + 1);
-        }
-        out.println(skip(step) + "]");
-    }
+	}
 
-    @Override
-    public void visit(DistinctClause dc, Integer step) throws AsterixException {
-        out.print(skip(step) + "Distinct ");
-        for (Expression expr : dc.getDistinctByExpr())
-            expr.accept(this, step + 1);
-    }
+	@Override
+	public void visit(UpdateStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(InsertStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(UpdateClause updateClause, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(DeleteStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(CreateDataverseStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+	}
 
-    @Override
-    public void visit(UpdateStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	@Override
+	public void visit(IndexDropStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(NodeGroupDropStatement deleteStatement, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+	}
 
-    @Override
-    public void visit(UpdateClause updateClause, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	@Override
+	public void visit(DataverseDropStatement deleteStatement, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(TypeDropStatement deleteStatement, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+	}
 
-    @Override
-    public void visit(CreateDataverseStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
-    }
+	@Override
+	public void visit(CreateFunctionStatement cfs, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(IndexDropStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
-    }
+	}
 
-    @Override
-    public void visit(NodeGroupDropStatement deleteStatement, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
-    }
+	@Override
+	public void visit(FunctionDropStatement fds, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(DataverseDropStatement deleteStatement, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
-    }
+	}
 
-    @Override
-    public void visit(TypeDropStatement deleteStatement, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
-    }
+	@Override
+	public void visit(CreateFeedStatement stmtDel, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(CreateFunctionStatement cfs, 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(FunctionDropStatement fds, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(FeedDropStatement stmt, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(BeginFeedStatement stmtDel, Integer arg) throws AsterixException {
-        // TODO Auto-generated method stub
+	}
 
-    }
+	@Override
+	public void visit(CompactStatement fds, Integer arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
 
-    @Override
-    public void visit(CompactStatement fds, 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 d9a92b4..4077cb6 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,18 +14,20 @@
  */
 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.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -67,102 +69,128 @@
 
 public interface IAqlExpressionVisitor<R, T> {
 
-    R visitQuery(Query q, T arg) throws AsterixException;
+	R visitQuery(Query q, T arg) throws AsterixException;
 
-    R visitFunctionDecl(FunctionDecl fd, T arg) throws AsterixException;
+	R visitFunctionDecl(FunctionDecl fd, T arg) throws AsterixException;
 
-    R visitTypeDecl(TypeDecl td, T arg) throws AsterixException;
+	R visitTypeDecl(TypeDecl td, T arg) throws AsterixException;
 
-    R visitNodegroupDecl(NodegroupDecl ngd, T arg) throws AsterixException;
+	R visitNodegroupDecl(NodegroupDecl ngd, T arg) throws AsterixException;
 
-    R visitDatasetDecl(DatasetDecl dd, T arg) throws AsterixException;
+	R visitDatasetDecl(DatasetDecl dd, T arg) throws AsterixException;
 
-    R visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, T arg) throws AsterixException;
+	R visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, T arg)
+			throws AsterixException;
 
-    R visitDropStatement(DropStatement del, T arg) throws AsterixException;
+	R visitDropStatement(DropStatement del, T arg) throws AsterixException;
 
-    R visitInsertStatement(InsertStatement insert, T arg) throws AsterixException;
+	R visitInsertStatement(InsertStatement insert, T arg)
+			throws AsterixException;
 
-    R visitDeleteStatement(DeleteStatement del, T arg) throws AsterixException;
+	R visitDeleteStatement(DeleteStatement del, T arg) throws AsterixException;
 
-    R visitUpdateStatement(UpdateStatement update, T arg) throws AsterixException;
+	R visitUpdateStatement(UpdateStatement update, T arg)
+			throws AsterixException;
 
-    R visitUpdateClause(UpdateClause del, T arg) throws AsterixException;
+	R visitUpdateClause(UpdateClause del, T arg) throws AsterixException;
 
-    R visitTypeReferenceExpression(TypeReferenceExpression tre, T arg) throws AsterixException;
+	R visitTypeReferenceExpression(TypeReferenceExpression tre, T arg)
+			throws AsterixException;
 
-    R visitRecordTypeDefiniton(RecordTypeDefinition tre, T arg) throws AsterixException;
+	R visitRecordTypeDefiniton(RecordTypeDefinition tre, T arg)
+			throws AsterixException;
 
-    R visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, T arg) throws AsterixException;
+	R visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, T arg)
+			throws AsterixException;
 
-    R visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, T arg) throws AsterixException;
+	R visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, T arg)
+			throws AsterixException;
 
-    R visitLiteralExpr(LiteralExpr l, T arg) throws AsterixException;
+	R visitLiteralExpr(LiteralExpr l, T arg) throws AsterixException;
 
-    R visitVariableExpr(VariableExpr v, T arg) throws AsterixException;
+	R visitVariableExpr(VariableExpr v, T arg) throws AsterixException;
 
-    R visitListConstructor(ListConstructor lc, T arg) throws AsterixException;
+	R visitListConstructor(ListConstructor lc, T arg) throws AsterixException;
 
-    R visitRecordConstructor(RecordConstructor rc, T arg) throws AsterixException;
+	R visitRecordConstructor(RecordConstructor rc, T arg)
+			throws AsterixException;
 
-    R visitOperatorExpr(OperatorExpr ifbo, T arg) throws AsterixException;
+	R visitOperatorExpr(OperatorExpr ifbo, T arg) throws AsterixException;
 
-    R visitFieldAccessor(FieldAccessor fa, T arg) throws AsterixException;
+	R visitFieldAccessor(FieldAccessor fa, T arg) throws AsterixException;
 
-    R visitIndexAccessor(IndexAccessor ia, T arg) throws AsterixException;
+	R visitIndexAccessor(IndexAccessor ia, T arg) throws AsterixException;
 
-    R visitIfExpr(IfExpr ifexpr, T arg) throws AsterixException;
+	R visitIfExpr(IfExpr ifexpr, T arg) throws AsterixException;
 
-    R visitFlworExpression(FLWOGRExpression flwor, T arg) throws AsterixException;
+	R visitFlworExpression(FLWOGRExpression flwor, T arg)
+			throws AsterixException;
 
-    R visitQuantifiedExpression(QuantifiedExpression qe, T arg) throws AsterixException;
+	R visitQuantifiedExpression(QuantifiedExpression qe, T arg)
+			throws AsterixException;
 
-    R visitForClause(ForClause fc, T arg) throws AsterixException;
+	R visitForClause(ForClause fc, T arg) throws AsterixException;
 
-    R visitLetClause(LetClause lc, T arg) throws AsterixException;
+	R visitLetClause(LetClause lc, T arg) throws AsterixException;
 
-    R visitWhereClause(WhereClause wc, T arg) throws AsterixException;
+	R visitWhereClause(WhereClause wc, T arg) throws AsterixException;
 
-    R visitOrderbyClause(OrderbyClause oc, T arg) throws AsterixException;
+	R visitOrderbyClause(OrderbyClause oc, T arg) throws AsterixException;
 
-    R visitGroupbyClause(GroupbyClause gc, T arg) throws AsterixException;
+	R visitGroupbyClause(GroupbyClause gc, T arg) throws AsterixException;
 
-    R visitLimitClause(LimitClause lc, T arg) throws AsterixException;
+	R visitLimitClause(LimitClause lc, T arg) throws AsterixException;
 
-    R visitDistinctClause(DistinctClause dc, T arg) throws AsterixException;
+	R visitDistinctClause(DistinctClause dc, T arg) throws AsterixException;
 
-    R visitUnaryExpr(UnaryExpr u, T arg) throws AsterixException;
+	R visitUnaryExpr(UnaryExpr u, T arg) throws AsterixException;
 
-    R visitUnionExpr(UnionExpr u, T arg) throws AsterixException;
+	R visitUnionExpr(UnionExpr u, T arg) throws AsterixException;
 
-    R visitCreateIndexStatement(CreateIndexStatement cis, T arg) throws AsterixException;
+	R visitCreateIndexStatement(CreateIndexStatement cis, T arg)
+			throws AsterixException;
 
-    R visitCreateDataverseStatement(CreateDataverseStatement del, T arg) throws AsterixException;
+	R visitCreateDataverseStatement(CreateDataverseStatement del, T arg)
+			throws AsterixException;
 
-    R visitIndexDropStatement(IndexDropStatement del, T arg) throws AsterixException;
+	R visitIndexDropStatement(IndexDropStatement del, T arg)
+			throws AsterixException;
 
-    R visitNodeGroupDropStatement(NodeGroupDropStatement del, T arg) throws AsterixException;
+	R visitNodeGroupDropStatement(NodeGroupDropStatement del, T arg)
+			throws AsterixException;
 
-    R visitDataverseDropStatement(DataverseDropStatement del, T arg) throws AsterixException;
+	R visitDataverseDropStatement(DataverseDropStatement del, T arg)
+			throws AsterixException;
 
-    R visitTypeDropStatement(TypeDropStatement del, T arg) throws AsterixException;
+	R visitTypeDropStatement(TypeDropStatement del, T arg)
+			throws AsterixException;
 
-    R visitWriteStatement(WriteStatement ws, T arg) throws AsterixException;
+	R visitWriteStatement(WriteStatement ws, T arg) throws AsterixException;
 
-    R visitSetStatement(SetStatement ss, T arg) throws AsterixException;
+	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 visitCallExpr(CallExpr pf, T arg) throws AsterixException;
+	R visitCreateFeedStatement(CreateFeedStatement del, T arg)
+			throws AsterixException;
 
-    R visitDataverseDecl(DataverseDecl dv, T arg) throws AsterixException;
+	R visitDropFeedStatement(FeedDropStatement del, T arg)
+			throws AsterixException;
 
-    R visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
+	R visitCallExpr(CallExpr pf, T arg) throws AsterixException;
 
-    R visitFunctionDropStatement(FunctionDropStatement del, T arg) throws AsterixException;
+	R visitDataverseDecl(DataverseDecl dv, T arg) throws AsterixException;
 
-    R visitCompactStatement(CompactStatement del, T arg) throws AsterixException;
+	R visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
+
+	R visitFunctionDropStatement(FunctionDropStatement del, T arg)
+			throws AsterixException;
+
+	R visitCompactStatement(CompactStatement del, 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 61a7183..79653c2 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,18 +14,20 @@
  */
 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.ConnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -155,9 +157,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 203758d..d0a4ed4 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,11 +24,12 @@
 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.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -38,6 +39,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;
@@ -92,467 +94,555 @@
 
 public final class AqlRewriter {
 
-    private final Query topExpr;
-    private final List<FunctionDecl> declaredFunctions;
-    private final AqlRewritingContext context;
-    private final MetadataTransactionContext mdTxnCtx;
-    private final AqlMetadataProvider metadataProvider;
+	private final Query topExpr;
+	private final List<FunctionDecl> declaredFunctions;
+	private final AqlRewritingContext context;
+	private final MetadataTransactionContext mdTxnCtx;
+	private final AqlMetadataProvider metadataProvider;
 
-    private enum DfsColor {
-        WHITE,
-        GRAY,
-        BLACK
-    }
+	private enum DfsColor {
+		WHITE, GRAY, BLACK
+	}
 
-    public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr, AqlMetadataProvider metadataProvider) {
-        this.topExpr = topExpr;
-        context = new AqlRewritingContext(topExpr.getVarCounter());
-        this.declaredFunctions = declaredFunctions;
-        this.mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        this.metadataProvider = metadataProvider;
-    }
+	public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr,
+			AqlMetadataProvider metadataProvider) {
+		this.topExpr = topExpr;
+		context = new AqlRewritingContext(topExpr.getVarCounter());
+		this.declaredFunctions = declaredFunctions;
+		this.mdTxnCtx = metadataProvider.getMetadataTxnContext();
+		this.metadataProvider = metadataProvider;
+	}
 
-    public Query getExpr() {
-        return topExpr;
-    }
+	public Query getExpr() {
+		return topExpr;
+	}
 
-    public int getVarCounter() {
-        return context.getVarCounter();
-    }
+	public int getVarCounter() {
+		return context.getVarCounter();
+	}
 
-    public void rewrite() throws AsterixException {
-        wrapInLets();
-        inlineDeclaredUdfs();
-    }
+	public void rewrite() throws AsterixException {
+		wrapInLets();
+		inlineDeclaredUdfs();
+	}
 
-    private void wrapInLets() {
-        // If the top expression of the main statement is not a FLWOR, it wraps
-        // it into a let clause.
-        if (topExpr == null) {
-            return;
-        }
-        Expression body = topExpr.getBody();
-        if (body.getKind() != Kind.FLWOGR_EXPRESSION) {
-            VarIdentifier var = context.newVariable();
-            VariableExpr v = new VariableExpr(var);
-            LetClause c1 = new LetClause(v, body);
-            ArrayList<Clause> clauseList = new ArrayList<Clause>(1);
-            clauseList.add(c1);
-            FLWOGRExpression newBody = new FLWOGRExpression(clauseList, new VariableExpr(var));
-            topExpr.setBody(newBody);
-        }
-    }
+	private void wrapInLets() {
+		// If the top expression of the main statement is not a FLWOR, it wraps
+		// it into a let clause.
+		if (topExpr == null) {
+			return;
+		}
+		Expression body = topExpr.getBody();
+		if (body.getKind() != Kind.FLWOGR_EXPRESSION) {
+			VarIdentifier var = context.newVariable();
+			VariableExpr v = new VariableExpr(var);
+			LetClause c1 = new LetClause(v, body);
+			ArrayList<Clause> clauseList = new ArrayList<Clause>(1);
+			clauseList.add(c1);
+			FLWOGRExpression newBody = new FLWOGRExpression(clauseList,
+					new VariableExpr(var));
+			topExpr.setBody(newBody);
+		}
+	}
 
-    private void inlineDeclaredUdfs() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
-        List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
-        for (FunctionDecl fdecl : declaredFunctions) {
-            funIds.add(fdecl.getSignature());
-        }
+	private void inlineDeclaredUdfs() throws AsterixException {
+		if (topExpr == null) {
+			return;
+		}
+		List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
+		for (FunctionDecl fdecl : declaredFunctions) {
+			funIds.add(fdecl.getSignature());
+		}
 
-        List<FunctionDecl> otherFDecls = new ArrayList<FunctionDecl>();
-        buildOtherUdfs(topExpr.getBody(), otherFDecls, funIds);
-        declaredFunctions.addAll(otherFDecls);
-        if (!declaredFunctions.isEmpty()) {
-            InlineUdfsVisitor visitor = new InlineUdfsVisitor(context);
-            while (topExpr.accept(visitor, declaredFunctions)) {
-                // loop until no more changes
-            }
-        }
-        declaredFunctions.removeAll(otherFDecls);
-    }
+		List<FunctionDecl> otherFDecls = new ArrayList<FunctionDecl>();
+		buildOtherUdfs(topExpr.getBody(), otherFDecls, funIds);
+		declaredFunctions.addAll(otherFDecls);
+		if (!declaredFunctions.isEmpty()) {
+			InlineUdfsVisitor visitor = new InlineUdfsVisitor(context);
+			while (topExpr.accept(visitor, declaredFunctions)) {
+				// loop until no more changes
+			}
+		}
+		declaredFunctions.removeAll(otherFDecls);
+	}
 
-    private void buildOtherUdfs(Expression expression, List<FunctionDecl> functionDecls,
-            List<FunctionSignature> declaredFunctions) throws AsterixException {
-        if (expression == null) {
-            return;
-        }
-        String value = metadataProvider.getConfig().get(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
-        boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
-        Set<FunctionSignature> functionCalls = getFunctionCalls(expression);
-        for (FunctionSignature signature : functionCalls) {
+	private void buildOtherUdfs(Expression expression,
+			List<FunctionDecl> functionDecls,
+			List<FunctionSignature> declaredFunctions) throws AsterixException {
+		if (expression == null) {
+			return;
+		}
+		String value = metadataProvider.getConfig().get(
+				FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
+		boolean includePrivateFunctions = (value != null) ? Boolean
+				.valueOf(value.toLowerCase()) : false;
+		Set<FunctionSignature> functionCalls = getFunctionCalls(expression);
+		for (FunctionSignature signature : functionCalls) {
 
-            if (declaredFunctions != null && declaredFunctions.contains(signature)) {
-                continue;
-            }
+			if (declaredFunctions != null
+					&& declaredFunctions.contains(signature)) {
+				continue;
+			}
 
-            FunctionDecl functionDecl = lookupUserDefinedFunctionDecl(signature);
-            if (functionDecl != null) {
-                if (functionDecls.contains(functionDecl)) {
-                    throw new AsterixException(" Detected recursvity!");
-                } else {
-                    functionDecls.add(functionDecl);
-                    buildOtherUdfs(functionDecl.getFuncBody(), functionDecls, declaredFunctions);
-                }
-            } else {
-                if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(signature, includePrivateFunctions)) {
-                    continue;
-                } else {
-                    throw new AsterixException(" unknown function " + signature);
-                }
-            }
-        }
-    }
+			Function function = lookupUserDefinedFunctionDecl(signature);
+			if (function == null) {
+				if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(
+						signature, includePrivateFunctions)) {
+					continue;
+				}
+				StringBuilder messageBuilder = new StringBuilder();
+				if (functionDecls.size() > 0) {
+					messageBuilder.append(" function "
+							+ functionDecls.get(functionDecls.size() - 1)
+									.getSignature() + " depends upon function "
+							+ signature + " which is undefined");
+				} else {
+					messageBuilder.append(" function " + signature
+							+ " is undefined ");
+				}
+				throw new AsterixException(messageBuilder.toString());
+			}
 
-    private FunctionDecl lookupUserDefinedFunctionDecl(FunctionSignature signature) throws AsterixException {
-        if (signature.getNamespace() == null) {
-            return null;
-        }
-        Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
-        if (function == null) {
-            return null;
-        }
-        return FunctionUtils.getFunctionDecl(function);
+			if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+				FunctionDecl functionDecl = FunctionUtils
+						.getFunctionDecl(function);
+				if (functionDecl != null) {
+					if (functionDecls.contains(functionDecl)) {
+						throw new AsterixException(
+								"ERROR:Recursive invocation "
+										+ functionDecls.get(
+												functionDecls.size() - 1)
+												.getSignature() + " <==> "
+										+ functionDecl.getSignature());
+					}
+					functionDecls.add(functionDecl);
+					buildOtherUdfs(functionDecl.getFuncBody(), functionDecls,
+							declaredFunctions);
+				}
+			}
+		}
 
-    }
+	}
 
-    private Set<FunctionSignature> getFunctionCalls(Expression expression) throws AsterixException {
-        Map<AsterixFunction, DfsColor> color = new HashMap<AsterixFunction, DfsColor>();
-        Map<AsterixFunction, List<AsterixFunction>> arcs = new HashMap<AsterixFunction, List<AsterixFunction>>();
-        GatherFunctionCalls gfc = new GatherFunctionCalls();
-        expression.accept(gfc, null);
-        return gfc.getCalls();
-    }
+	private Function lookupUserDefinedFunctionDecl(FunctionSignature signature)
+			throws AsterixException {
+		if (signature.getNamespace() == null) {
+			return null;
+		}
+		return MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
+	}
 
-    private static class GatherFunctionCalls implements IAqlExpressionVisitor<Void, Void> {
+	private Set<FunctionSignature> getFunctionCalls(Expression expression)
+			throws AsterixException {
+		Map<AsterixFunction, DfsColor> color = new HashMap<AsterixFunction, DfsColor>();
+		Map<AsterixFunction, List<AsterixFunction>> arcs = new HashMap<AsterixFunction, List<AsterixFunction>>();
+		GatherFunctionCalls gfc = new GatherFunctionCalls();
+		expression.accept(gfc, null);
+		return gfc.getCalls();
+	}
 
-        private final Set<FunctionSignature> calls = new HashSet<FunctionSignature>();
+	private static class GatherFunctionCalls implements
+			IAqlExpressionVisitor<Void, Void> {
 
-        public GatherFunctionCalls() {
-        }
+		private final Set<FunctionSignature> calls = new HashSet<FunctionSignature>();
 
-        @Override
-        public Void visitCallExpr(CallExpr pf, Void arg) throws AsterixException {
-            calls.add(pf.getFunctionSignature());
-            for (Expression e : pf.getExprList()) {
-                e.accept(this, arg);
-            }
-            return null;
-        }
+		public GatherFunctionCalls() {
+		}
 
-        @Override
-        public Void visitCreateIndexStatement(CreateIndexStatement cis, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitCallExpr(CallExpr pf, Void arg)
+				throws AsterixException {
+			calls.add(pf.getFunctionSignature());
+			for (Expression e : pf.getExprList()) {
+				e.accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitDataverseDecl(DataverseDecl dv, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitCreateIndexStatement(CreateIndexStatement cis, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitDeleteStatement(DeleteStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDataverseDecl(DataverseDecl dv, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitDistinctClause(DistinctClause dc, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDeleteStatement(DeleteStatement del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitDropStatement(DropStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDistinctClause(DistinctClause dc, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitDatasetDecl(DatasetDecl dd, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDropStatement(DropStatement del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitFieldAccessor(FieldAccessor fa, Void arg) throws AsterixException {
-            fa.getExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitDatasetDecl(DatasetDecl dd, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitFlworExpression(FLWOGRExpression flwor, Void arg) throws AsterixException {
-            for (Clause c : flwor.getClauseList()) {
-                c.accept(this, arg);
-            }
-            flwor.getReturnExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitFieldAccessor(FieldAccessor fa, Void arg)
+				throws AsterixException {
+			fa.getExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitForClause(ForClause fc, Void arg) throws AsterixException {
-            fc.getInExpr().accept(this, arg);
-            if (fc.getPosVarExpr() != null) {
-                fc.getPosVarExpr().accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitFlworExpression(FLWOGRExpression flwor, Void arg)
+				throws AsterixException {
+			for (Clause c : flwor.getClauseList()) {
+				c.accept(this, arg);
+			}
+			flwor.getReturnExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitFunctionDecl(FunctionDecl fd, Void arg) throws AsterixException {
-            fd.getFuncBody().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitForClause(ForClause fc, Void arg)
+				throws AsterixException {
+			fc.getInExpr().accept(this, arg);
+			if (fc.getPosVarExpr() != null) {
+				fc.getPosVarExpr().accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitGroupbyClause(GroupbyClause gc, Void arg) throws AsterixException {
-            for (GbyVariableExpressionPair p : gc.getGbyPairList()) {
-                p.getExpr().accept(this, arg);
-            }
-            for (GbyVariableExpressionPair p : gc.getDecorPairList()) {
-                p.getExpr().accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitFunctionDecl(FunctionDecl fd, Void arg)
+				throws AsterixException {
+			fd.getFuncBody().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitIfExpr(IfExpr ifexpr, Void arg) throws AsterixException {
-            ifexpr.getCondExpr().accept(this, arg);
-            ifexpr.getThenExpr().accept(this, arg);
-            ifexpr.getElseExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitGroupbyClause(GroupbyClause gc, Void arg)
+				throws AsterixException {
+			for (GbyVariableExpressionPair p : gc.getGbyPairList()) {
+				p.getExpr().accept(this, arg);
+			}
+			for (GbyVariableExpressionPair p : gc.getDecorPairList()) {
+				p.getExpr().accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitIndexAccessor(IndexAccessor ia, Void arg) throws AsterixException {
-            ia.getExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitIfExpr(IfExpr ifexpr, Void arg)
+				throws AsterixException {
+			ifexpr.getCondExpr().accept(this, arg);
+			ifexpr.getThenExpr().accept(this, arg);
+			ifexpr.getElseExpr().accept(this, arg);
+			return null;
+		}
+
+		@Override
+		public Void visitIndexAccessor(IndexAccessor ia, Void arg)
+				throws AsterixException {
+			ia.getExpr().accept(this, arg);
+			return null;
+		}
+
+		@Override
+		public Void visitInsertStatement(InsertStatement insert, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
+
+		@Override
+		public Void visitLetClause(LetClause lc, Void arg)
+				throws AsterixException {
+			lc.getBindingExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitInsertStatement(InsertStatement insert, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitLimitClause(LimitClause lc, Void arg)
+				throws AsterixException {
+			lc.getLimitExpr().accept(this, arg);
+			if (lc.getOffset() != null) {
+				lc.getOffset().accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitLetClause(LetClause lc, Void arg) throws AsterixException {
-            lc.getBindingExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitListConstructor(ListConstructor lc, Void arg)
+				throws AsterixException {
+			for (Expression e : lc.getExprList()) {
+				e.accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitLimitClause(LimitClause lc, Void arg) throws AsterixException {
-            lc.getLimitExpr().accept(this, arg);
-            if (lc.getOffset() != null) {
-                lc.getOffset().accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitLiteralExpr(LiteralExpr l, Void arg)
+				throws AsterixException {
+			// do nothing
+			return null;
+		}
 
-        @Override
-        public Void visitListConstructor(ListConstructor lc, Void arg) throws AsterixException {
-            for (Expression e : lc.getExprList()) {
-                e.accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitLiteralExpr(LiteralExpr l, Void arg) throws AsterixException {
-            // do nothing
-            return null;
-        }
+		@Override
+		public Void visitNodegroupDecl(NodegroupDecl ngd, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitOperatorExpr(OperatorExpr op, Void arg)
+				throws AsterixException {
+			for (Expression e : op.getExprList()) {
+				e.accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitNodegroupDecl(NodegroupDecl ngd, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitOrderbyClause(OrderbyClause oc, Void arg)
+				throws AsterixException {
+			for (Expression e : oc.getOrderbyList()) {
+				e.accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitOperatorExpr(OperatorExpr op, Void arg) throws AsterixException {
-            for (Expression e : op.getExprList()) {
-                e.accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitOrderedListTypeDefiniton(
+				OrderedListTypeDefinition olte, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitOrderbyClause(OrderbyClause oc, Void arg) throws AsterixException {
-            for (Expression e : oc.getOrderbyList()) {
-                e.accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitQuantifiedExpression(QuantifiedExpression qe, Void arg)
+				throws AsterixException {
+			for (QuantifiedPair qp : qe.getQuantifiedList()) {
+				qp.getExpr().accept(this, arg);
+			}
+			qe.getSatisfiesExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitQuery(Query q, Void arg) throws AsterixException {
+			q.getBody().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitQuantifiedExpression(QuantifiedExpression qe, Void arg) throws AsterixException {
-            for (QuantifiedPair qp : qe.getQuantifiedList()) {
-                qp.getExpr().accept(this, arg);
-            }
-            qe.getSatisfiesExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitRecordConstructor(RecordConstructor rc, Void arg)
+				throws AsterixException {
+			for (FieldBinding fb : rc.getFbList()) {
+				fb.getLeftExpr().accept(this, arg);
+				fb.getRightExpr().accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitQuery(Query q, Void arg) throws AsterixException {
-            q.getBody().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitRecordTypeDefiniton(RecordTypeDefinition tre, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitRecordConstructor(RecordConstructor rc, Void arg) throws AsterixException {
-            for (FieldBinding fb : rc.getFbList()) {
-                fb.getLeftExpr().accept(this, arg);
-                fb.getRightExpr().accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitSetStatement(SetStatement ss, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitRecordTypeDefiniton(RecordTypeDefinition tre, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitTypeDecl(TypeDecl td, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitSetStatement(SetStatement ss, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitTypeReferenceExpression(TypeReferenceExpression tre,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitTypeDecl(TypeDecl td, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitUnaryExpr(UnaryExpr u, Void arg)
+				throws AsterixException {
+			u.getExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitTypeReferenceExpression(TypeReferenceExpression tre, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitUnionExpr(UnionExpr u, Void arg)
+				throws AsterixException {
+			for (Expression e : u.getExprs()) {
+				e.accept(this, arg);
+			}
+			return null;
+		}
 
-        @Override
-        public Void visitUnaryExpr(UnaryExpr u, Void arg) throws AsterixException {
-            u.getExpr().accept(this, arg);
-            return null;
-        }
+		@Override
+		public Void visitUnorderedListTypeDefiniton(
+				UnorderedListTypeDefinition ulte, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitUnionExpr(UnionExpr u, Void arg) throws AsterixException {
-            for (Expression e : u.getExprs()) {
-                e.accept(this, arg);
-            }
-            return null;
-        }
+		@Override
+		public Void visitUpdateClause(UpdateClause del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitUpdateStatement(UpdateStatement update, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitUpdateClause(UpdateClause del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitVariableExpr(VariableExpr v, Void arg)
+				throws AsterixException {
+			// do nothing
+			return null;
+		}
 
-        @Override
-        public Void visitUpdateStatement(UpdateStatement update, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitWhereClause(WhereClause wc, Void arg)
+				throws AsterixException {
+			wc.getWhereExpr().accept(this, arg);
+			return null;
+		}
 
-        @Override
-        public Void visitVariableExpr(VariableExpr v, Void arg) throws AsterixException {
-            // do nothing
-            return null;
-        }
+		@Override
+		public Void visitWriteStatement(WriteStatement ws, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitWhereClause(WhereClause wc, Void arg) throws AsterixException {
-            wc.getWhereExpr().accept(this, arg);
-            return null;
-        }
+		public Set<FunctionSignature> getCalls() {
+			return calls;
+		}
 
-        @Override
-        public Void visitWriteStatement(WriteStatement ws, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitCreateDataverseStatement(CreateDataverseStatement del,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        public Set<FunctionSignature> getCalls() {
-            return calls;
-        }
+		@Override
+		public Void visitIndexDropStatement(IndexDropStatement del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitCreateDataverseStatement(CreateDataverseStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitNodeGroupDropStatement(NodeGroupDropStatement del,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitIndexDropStatement(IndexDropStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDataverseDropStatement(DataverseDropStatement del,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitNodeGroupDropStatement(NodeGroupDropStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitTypeDropStatement(TypeDropStatement del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitDataverseDropStatement(DataverseDropStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitDisconnectFeedStatement(DisconnectFeedStatement del,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitTypeDropStatement(TypeDropStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visit(CreateFunctionStatement cfs, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitControlFeedStatement(ControlFeedStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitFunctionDropStatement(FunctionDropStatement del,
+				Void arg) throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visit(CreateFunctionStatement cfs, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitCreateFeedStatement(CreateFeedStatement del, Void arg)
+				throws AsterixException {
+			// TODO Auto-generated method stub
+			return null;
+		}
 
-        @Override
-        public Void visitFunctionDropStatement(FunctionDropStatement 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 visitBeginFeedStatement(BeginFeedStatement bf, 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;
+		}
 
-        @Override
-        public Void visitCompactStatement(CompactStatement del, Void arg) throws AsterixException {
-            // TODO Auto-generated method stub
-            return null;
-        }
+		@Override
+		public Void visitCompactStatement(CompactStatement 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 d860b0c..1ec5f67 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,11 +21,12 @@
 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.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -410,7 +412,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;
@@ -578,7 +580,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 fc7acfe..1fed1ec 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,11 +22,12 @@
 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.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 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;
@@ -36,6 +37,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;
@@ -500,7 +502,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;
     }
@@ -519,7 +522,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 9080f9c..8b46dc2 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -86,6 +86,12 @@
       }
     };  
    
+    private static class FunctionName {
+	   public String dataverse = null;
+	   public String library = null;
+	   public String function = null;
+	}
+
     private static String getHint(Token t) {
         if (t.specialToken == null) {
             return null;
@@ -234,6 +240,7 @@
     | stmt = IndexSpecification()
     | stmt = DataverseSpecification()
     | stmt = FunctionSpecification()
+    | stmt = FeedSpecification()
   )        
   {
     return stmt;
@@ -247,7 +254,7 @@
   TypeExpression typeExpr = null;
 }
 {
-  "type" nameComponents = FunctionOrTypeName() ifNotExists = IfNotExists()
+  "type" nameComponents = TypeName() ifNotExists = IfNotExists()
   "as" typeExpr = TypeExpr()
     {
       long numValues = -1;
@@ -325,32 +332,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() )?
-    ( "using" "compaction" "policy" compactionPolicy = CompactionPolicy() compactionPolicyProperties = Configuration() )?
-      {
-        FeedDetailsDecl fdd = new FeedDetailsDecl(adapterName,
-                                                  properties,
-                                                  appliedFunction,
-                                                  nodeGroupName != null
-                                                    ? new Identifier(nodeGroupName)
-                                                    : null,
-                                                  primaryKeyFields,
-                                                  compactionPolicy,
-                                                  compactionPolicyProperties);
-        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()
@@ -474,33 +455,61 @@
   boolean ifNotExists = false;
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
   String functionBody;
+  VarIdentifier var = null;
   Expression functionBodyExpr;
   Token beginPos;
   Token endPos;
-  Pair<Identifier,Identifier> nameComponents=null;
-
+  FunctionName fctName = null;
+  
   createNewScope();
 }
 {
-  "function" nameComponents = FunctionOrTypeName()
+  "function" fctName = FunctionName()
   ifNotExists = IfNotExists()
   paramList = ParameterList()
   <LEFTBRACE>
-    {
-      beginPos = token;
-    } 
+  {
+     beginPos = token;
+  } 
   functionBodyExpr = Expression() <RIGHTBRACE>
     {
       endPos = token;
       functionBody = extractFragment(beginPos.beginLine, beginPos.beginColumn, endPos.beginLine, endPos.beginColumn);
-      String dataverse = nameComponents.first.getValue();
-      String functionName = nameComponents.second.getValue();      
-      signature = new FunctionSignature(dataverse, functionName, paramList.size());
+      // TODO use fctName.library
+      signature = new FunctionSignature(fctName.dataverse, fctName.function, paramList.size());
       getCurrentScope().addFunctionDescriptor(signature, false);
       return new CreateFunctionStatement(signature, paramList, functionBody, ifNotExists);
     }
 }
 
+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>();
@@ -552,22 +561,34 @@
     }
 }
 
+String GetPolicy() throws ParseException:
+{
+  String policy = null;
+}
+{
+   "using" "policy" policy = Identifier() 
+   {
+     return policy;
+   }
+   
+}   
+
 FunctionSignature FunctionSignature() throws ParseException:
 {
-  Pair<Identifier,Identifier> pairId = null;
+  FunctionName fctName = null;
   int arity = 0;
 }
 {
-  pairId = FunctionOrTypeName() "@" <INTEGER_LITERAL> 
+  fctName = FunctionName() "@" <INTEGER_LITERAL> 
     {  
       arity = new Integer(token.image);
       if (arity < 0 && arity != FunctionIdentifier.VARARGS) {
         throw new ParseException(" invalid arity:" + arity);
       }
 
-      String dataverse = pairId.first.getValue();
-      String functionName = pairId.second.getValue(); 
-      return new FunctionSignature(dataverse, functionName, arity);
+      // TODO use fctName.library
+      String fqFunctionName = fctName.library == null ? fctName.function : fctName.library + "#" + fctName.function;
+      return new FunctionSignature(fctName.dataverse, fqFunctionName, arity);
     }
 }
 
@@ -615,7 +636,7 @@
       {
         stmt = new NodeGroupDropStatement(new Identifier(id), ifExists);
       }
-    | "type" pairId = FunctionOrTypeName() ifExists = IfExists()
+    | "type" pairId = TypeName() ifExists = IfExists()
       {
         stmt = new TypeDropStatement(pairId.first, pairId.second, ifExists);
       }
@@ -627,6 +648,10 @@
       {
         stmt = new FunctionDropStatement(funcSig, ifExists);
       }
+    | "feed" pairId = QualifiedName() ifExists = IfExists()
+      {
+        stmt = new FeedDropStatement(pairId.first, pairId.second, ifExists);
+      }  
   )
   {
     return stmt;  
@@ -809,31 +834,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()
+    "connect" "feed" feedNameComponents = QualifiedName() "to" <DATASET> datasetNameComponents = QualifiedName() (policy = GetPolicy())? 
       {
-        stmt = new BeginFeedStatement(nameComponents.first, nameComponents.second, 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);
       }
   )
     {
@@ -1026,8 +1042,46 @@
   }
 }
 
+FunctionName FunctionName() throws ParseException:
+{
+  String first = null;
+  String second = null;
+  String third = null;
+  boolean secondAfterDot = false;
+}
+{
+  first = Identifier() ( "." second = Identifier()
+    {
+      secondAfterDot = true;
+    }
+  ("#" third = Identifier())? | "#" second = Identifier() )?
+    {
+      FunctionName result = new FunctionName();
+      if (second == null) {
+        result.dataverse = defaultDataverse;
+        result.library   = null;
+        result.function  = first;
+      } else if (third == null) {
+        if (secondAfterDot) {
+          result.dataverse = first;
+          result.library   = null;
+          result.function  = second;
+        } else {
+          result.dataverse = defaultDataverse;
+          result.library   = first;
+          result.function  = second;
+        }
+      } else {
+        result.dataverse = first;
+        result.library   = second;
+        result.function  = third;
+      }
+      return result;
+    }  
+}
 
-Pair<Identifier,Identifier> FunctionOrTypeName() throws ParseException:
+
+Pair<Identifier,Identifier> TypeName() throws ParseException:
 {
   Pair<Identifier,Identifier> name = null;
 }
@@ -1684,18 +1738,12 @@
   List<Expression> argList = new ArrayList<Expression>();
   Expression tmp;
   int arity = 0;
-  Pair<Identifier,Identifier> funcId = null;
-  String funcName;
-  String dataverse;
+  FunctionName funcName = null;
   String hint = null;
-  String id1 = null;
-  String id2 = null;
 }
 {  
-  funcId = FunctionOrTypeName()
+  funcName = FunctionName()
     {
-      dataverse = funcId.first.getValue();
-      funcName = funcId.second.getValue();
       hint = getHint(token);
     }
   <LEFTPAREN> (tmp = Expression()
@@ -1710,9 +1758,12 @@
     }
   )*)? <RIGHTPAREN>
     {
-      FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, arity);
+      // TODO use funcName.library
+      String fqFunctionName = funcName.library == null ? funcName.function : funcName.library + "#" + funcName.function;
+      FunctionSignature signature
+        = lookupFunctionSignature(funcName.dataverse, fqFunctionName, arity);
       if (signature == null) {
-        signature = new FunctionSignature(dataverse, funcName, arity);
+        signature = new FunctionSignature(funcName.dataverse, fqFunctionName, arity);
       }
       callExpr = new CallExpr(signature,argList);
       if (hint != null && hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
@@ -1722,6 +1773,7 @@
     }
 }
 
+
 Expression DatasetAccessExpression() throws ParseException:
 {
   String funcName;
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 7d1786e..8697ef3 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -70,6 +66,40 @@
 							<generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
 						</configuration>
 					</execution>
+					<execution>
+						<id>cluster</id>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+						<configuration>
+							<args>
+								<arg>-Xsetters</arg>
+								<arg>-Xvalue-constructor</arg>
+							</args>
+							<plugins>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-basics</artifactId>
+									<version>0.6.2</version>
+								</plugin>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-value-constructor</artifactId>
+									<version>3.0</version>
+								</plugin>
+							</plugins>
+							<schemaDirectory>src/main/resources/schema</schemaDirectory>
+							<schemaIncludes>
+								<include>cluster.xsd</include>
+							</schemaIncludes>
+							<generatePackage>edu.uci.ics.asterix.event.schema.cluster</generatePackage>
+							<generateDirectory>${project.build.directory}/generated-sources/cluster</generateDirectory>
+							<bindingDirectory>src/main/resources/schema</bindingDirectory>
+							<bindingIncludes>
+								<bindingInclude>jaxb-bindings.xjb</bindingInclude>
+							</bindingIncludes>
+						</configuration>
+					</execution>
 				</executions>
 			</plugin>
 			<plugin>
@@ -94,6 +124,16 @@
 
 	<dependencies>
 		<dependency>
+			<groupId>commons-io</groupId>
+			<artifactId>commons-io</artifactId>
+			<version>1.4</version>
+		</dependency>
+		<dependency>
+			<groupId>commons-httpclient</groupId>
+			<artifactId>commons-httpclient</artifactId>
+			<version>3.0.1</version>
+		</dependency>
+		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>algebricks-compiler</artifactId>
 		</dependency>
@@ -116,6 +156,14 @@
 			<artifactId>hyracks-storage-am-lsm-common</artifactId>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-common</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-api</artifactId>
+		</dependency>
+		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
 			<artifactId>asterix-test-framework</artifactId>
 			<version>0.8.1-SNAPSHOT</version>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
index e975f60..63a8057 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
@@ -27,6 +27,15 @@
     private static final String EXTERNAL_APISERVER_KEY = "api.port";
     private static int EXTERNAL_APISERVER_DEFAULT = 19002;
 
+    private static final String EXTERNAL_FEEDSERVER_KEY = "feed.port";
+    private static int EXTERNAL_FEEDSERVER_DEFAULT = 19003;
+
+    private static final String EXTERNAL_CC_JAVA_OPTS_KEY = "cc.java.opts";
+    private static String EXTERNAL_CC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+
+    private static final String EXTERNAL_NC_JAVA_OPTS_KEY = "nc.java.opts";
+    private static String EXTERNAL_NC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+
     public AsterixExternalProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
     }
@@ -41,8 +50,23 @@
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    public int getFeedServerPort() {
+        return accessor.getProperty(EXTERNAL_FEEDSERVER_KEY, EXTERNAL_FEEDSERVER_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
     public Level getLogLevel() {
         return accessor.getProperty(EXTERNAL_LOGLEVEL_KEY, EXTERNAL_LOGLEVEL_DEFAULT,
                 PropertyInterpreters.getLevelPropertyInterpreter());
     }
+
+    public String getNCJavaParams() {
+        return accessor.getProperty(EXTERNAL_NC_JAVA_OPTS_KEY, EXTERNAL_NC_JAVA_OPTS_DEFAULT,
+                PropertyInterpreters.getStringPropertyInterpreter());
+    }
+
+    public String getCCJavaParams() {
+        return accessor.getProperty(EXTERNAL_CC_JAVA_OPTS_KEY, EXTERNAL_CC_JAVA_OPTS_DEFAULT,
+                PropertyInterpreters.getStringPropertyInterpreter());
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
index 323f5f8..1d68dfc 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
@@ -23,6 +23,10 @@
         super(accessor);
     }
 
+    public String getInstanceName() {
+        return accessor.getInstanceName();
+    }
+
     public String getMetadataNodeName() {
         return accessor.getMetadataNodeName();
     }
@@ -38,9 +42,12 @@
     public Set<String> getNodeNames() {
         return accessor.getNodeNames();
     }
-    
-    public String getCoredumpPath(String nodeId){
+
+    public String getCoredumpPath(String nodeId) {
         return accessor.getCoredumpPath(nodeId);
     }
 
+    public Map<String, String> getCoredumpPaths() {
+        return accessor.getCoredumpConfig();
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
index 2bd292e..7654aa3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
@@ -39,6 +39,7 @@
 public class AsterixPropertiesAccessor {
     private static final Logger LOGGER = Logger.getLogger(AsterixPropertiesAccessor.class.getName());
 
+    private final String instanceName;
     private final String metadataNodeName;
     private final Set<String> nodeNames;
     private final Map<String, String[]> stores;
@@ -69,6 +70,7 @@
         } catch (JAXBException e) {
             throw new AsterixException("Failed to read configuration file " + fileName);
         }
+        instanceName = asterixConfiguration.getInstanceName();
         metadataNodeName = asterixConfiguration.getMetadataNode();
         stores = new HashMap<String, String[]>();
         List<Store> configuredStores = asterixConfiguration.getStore();
@@ -113,8 +115,26 @@
         return coredumpConfig.get(nodeId);
     }
 
-    public String getTransactionLogDir(String nodeId) {
-        return transactionLogDirs.get(nodeId);
+    public Map<String, String> getTransactionLogDirs() {
+        return transactionLogDirs;
+    }
+
+    public Map<String, String> getCoredumpConfig() {
+        return coredumpConfig;
+    }
+
+    public void putCoredumpPaths(String nodeId, String coredumpPath) {
+        if (coredumpConfig.containsKey(nodeId)) {
+            throw new IllegalStateException("Cannot override value for coredump path");
+        }
+        coredumpConfig.put(nodeId, coredumpPath);
+    }
+
+    public void putTransactionLogDir(String nodeId, String txnLogDir) {
+        if (transactionLogDirs.containsKey(nodeId)) {
+            throw new IllegalStateException("Cannot override value for txnLogDir");
+        }
+        transactionLogDirs.put(nodeId, txnLogDir);
     }
 
     public <T> T getProperty(String property, T defaultValue, IPropertyInterpreter<T> interpreter) {
@@ -137,4 +157,9 @@
                     + "'.\n See the description: \n" + p.getDescription() + "\nDefault = " + defaultValue);
         }
     }
+
+    public String getInstanceName() {
+        return instanceName;
+    }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index a1dd52a..978577a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.common.config;
 
+import java.util.Map;
+
 public class AsterixTransactionProperties extends AbstractAsterixProperties {
 
     private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
@@ -51,7 +53,11 @@
     }
 
     public String getLogDirectory(String nodeId) {
-        return accessor.getTransactionLogDir(nodeId);
+        return accessor.getTransactionLogDirs().get(nodeId);
+    }
+
+    public Map<String, String> getLogDirectories() {
+        return accessor.getTransactionLogDirs();
     }
 
     public int getLogBufferNumPages() {
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-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index bc19543..26a79c3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -75,6 +75,7 @@
                 }
             }
         } catch (Exception e) {
+            e.printStackTrace();
             throw new HyracksDataException(e);
         }
         System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
index e58bbfa..6f6a7b2 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
@@ -56,4 +56,19 @@
                 recordDescProvider, op, isPrimary);
     }
 
+    public boolean isPrimary() {
+        return isPrimary;
+    }
+
+    public int[] getFieldPermutations() {
+        return fieldPermutation;
+    }
+
+    public IndexOperation getIndexOperation() {
+        return op;
+    }
+
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        return comparatorFactories;
+    }
 }
diff --git a/asterix-common/src/main/resources/schema/asterix-conf.xsd b/asterix-common/src/main/resources/schema/asterix-conf.xsd
index f461723..3033893 100644
--- a/asterix-common/src/main/resources/schema/asterix-conf.xsd
+++ b/asterix-common/src/main/resources/schema/asterix-conf.xsd
@@ -5,7 +5,8 @@
 
 	<!-- definition of simple types -->
 
-        
+
+	<xs:element name="instanceName" type="xs:string" />
 	<xs:element name="metadataNode" type="xs:string" />
 	<xs:element name="coredumpPath" type="xs:string" />
 	<xs:element name="storeDirs" type="xs:string" />
@@ -13,9 +14,9 @@
 	<xs:element name="name" type="xs:string" />
 	<xs:element name="value" type="xs:string" />
 	<xs:element name="description" type="xs:string" />
-    <xs:element name="txnLogDirPath" type="xs:string" />
-	
-	
+	<xs:element name="txnLogDirPath" type="xs:string" />
+
+
 	<!-- definition of complex elements -->
 	<xs:element name="store">
 		<xs:complexType>
@@ -34,7 +35,7 @@
 			</xs:sequence>
 		</xs:complexType>
 	</xs:element>
-	
+
 	<xs:element name="transactionLogDir">
 		<xs:complexType>
 			<xs:sequence>
@@ -58,10 +59,11 @@
 	<xs:element name="asterixConfiguration">
 		<xs:complexType>
 			<xs:sequence>
-				<xs:element ref="mg:metadataNode" minOccurs="0"/>
+				<xs:element ref="mg:instanceName" minOccurs="0" />
+				<xs:element ref="mg:metadataNode" minOccurs="0" />
 				<xs:element ref="mg:store" maxOccurs="unbounded" />
 				<xs:element ref="mg:coredump" maxOccurs="unbounded" />
-				<xs:element ref="mg:transactionLogDir" maxOccurs="unbounded"/>
+				<xs:element ref="mg:transactionLogDir" maxOccurs="unbounded" />
 				<xs:element ref="mg:property" minOccurs="0" maxOccurs="unbounded" />
 			</xs:sequence>
 		</xs:complexType>
diff --git a/asterix-events/src/main/resources/schema/cluster.xsd b/asterix-common/src/main/resources/schema/cluster.xsd
similarity index 79%
rename from asterix-events/src/main/resources/schema/cluster.xsd
rename to asterix-common/src/main/resources/schema/cluster.xsd
index f0d5bd9..8c317d8 100644
--- a/asterix-events/src/main/resources/schema/cluster.xsd
+++ b/asterix-common/src/main/resources/schema/cluster.xsd
@@ -3,7 +3,8 @@
 	xmlns:cl="cluster" targetNamespace="cluster" elementFormDefault="qualified">
 
 	<!-- definition of simple types -->
-	<xs:element name="name" type="xs:string" />
+	<xs:element name="instance_name" type="xs:string" />
+	<xs:element name="cluster_name" type="xs:string" />
 	<xs:element name="log_dir" type="xs:string" />
 	<xs:element name="txn_log_dir" type="xs:string" />
 	<xs:element name="id" type="xs:string" />
@@ -21,6 +22,9 @@
 	<xs:element name="client_port" type="xs:integer" />
 	<xs:element name="cluster_port" type="xs:integer" />
 	<xs:element name="http_port" type="xs:integer" />
+	<xs:element name="debug_port" type="xs:integer" />
+	<xs:element name="metadata_node" type="xs:string" />
+	
 
 	<!-- definition of complex elements -->
 	<xs:element name="working_dir">
@@ -40,9 +44,10 @@
 				<xs:element ref="cl:cluster_ip" />
 				<xs:element ref="cl:java_home" minOccurs="0" />
 				<xs:element ref="cl:log_dir" minOccurs="0" />
-				<xs:element ref="cl:client_port"  />
-				<xs:element ref="cl:cluster_port"  />
-				<xs:element ref="cl:http_port"  />
+				<xs:element ref="cl:client_port" />
+				<xs:element ref="cl:cluster_port" />
+				<xs:element ref="cl:http_port" />
+				<xs:element ref="cl:debug_port" minOccurs="0" />
 			</xs:sequence>
 		</xs:complexType>
 	</xs:element>
@@ -74,6 +79,15 @@
 				<xs:element ref="cl:txn_log_dir" minOccurs="0" />
 				<xs:element ref="cl:store" minOccurs="0" />
 				<xs:element ref="cl:iodevices" minOccurs="0" />
+				<xs:element ref="cl:debug_port" minOccurs="0" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
+
+	<xs:element name="substitute_nodes">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="cl:node" maxOccurs="unbounded" />
 			</xs:sequence>
 		</xs:complexType>
 	</xs:element>
@@ -81,7 +95,8 @@
 	<xs:element name="cluster">
 		<xs:complexType>
 			<xs:sequence>
-				<xs:element ref="cl:name" />
+				<xs:element ref="cl:instance_name" />
+				<xs:element ref="cl:cluster_name" />
 				<xs:element ref="cl:username" />
 				<xs:element ref="cl:env" minOccurs="0" />
 				<xs:element ref="cl:java_home" minOccurs="0" />
@@ -90,8 +105,10 @@
 				<xs:element ref="cl:store" minOccurs="0" />
 				<xs:element ref="cl:iodevices" minOccurs="0" />
 				<xs:element ref="cl:working_dir" />
+				<xs:element ref="cl:metadata_node" />
 				<xs:element ref="cl:master_node" />
 				<xs:element ref="cl:node" maxOccurs="unbounded" />
+				<xs:element ref="cl:substitute_nodes" />
 			</xs:sequence>
 		</xs:complexType>
 	</xs:element>
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java
new file mode 100644
index 0000000..5777d0a
--- /dev/null
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.asterix.test.aql;
+
+import java.util.List;
+
+public final class TestHelper {
+
+    public static boolean isInPrefixList(List<String> prefixList, String s) {
+        for (String s2 : prefixList) {
+            if (s.startsWith(s2)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+}
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 861cce5..d41815d 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -366,11 +366,15 @@
                         case "mgx":
                             executeManagixCommand(statement);
                             break;
+                        case "sleep":
+                            Thread.sleep(Long.parseLong(statement.trim()));
+                            break;
                         default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
 
                 } catch (Exception e) {
+                    e.printStackTrace();
                     if (cUnit.getExpectedError().isEmpty()) {
                         throw new Exception("Test \"" + testFile + "\" FAILED!", e);
                     }
diff --git a/asterix-events/pom.xml b/asterix-events/pom.xml
index f221df2..8457198 100644
--- a/asterix-events/pom.xml
+++ b/asterix-events/pom.xml
@@ -1,196 +1,207 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-          <artifactId>asterix</artifactId>
-          <groupId>edu.uci.ics.asterix</groupId>
-          <version>0.8.1-SNAPSHOT</version>
-  </parent>
-  <groupId>edu.uci.ics.asterix</groupId>
-  <artifactId>asterix-events</artifactId>
-  <version>0.8.1-SNAPSHOT</version>
+<!-- ! 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. ! -->
 
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<artifactId>asterix</artifactId>
+		<groupId>edu.uci.ics.asterix</groupId>
+		<version>0.8.1-SNAPSHOT</version>
+	</parent>
+	<groupId>edu.uci.ics.asterix</groupId>
+	<artifactId>asterix-events</artifactId>
+	<version>0.8.1-SNAPSHOT</version>
 
-  <build>
-    <plugins>
-       <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-compiler-plugin</artifactId>
-          <version>2.0.2</version>
-          <configuration>
-             <source>1.6</source>
-             <target>1.6</target>
-          </configuration>
-       </plugin>
-       <plugin>
-          <groupId>org.jvnet.jaxb2.maven2</groupId>
-          <artifactId>maven-jaxb2-plugin</artifactId>
-          <executions>
-            <execution>
-              <id>event</id>
-              <goals>
-                <goal>generate</goal>
-              </goals>
-              <configuration>
-                <args>
-                  <arg>-Xsetters</arg>
-                  <arg>-Xvalue-constructor</arg>
-                </args>
-                <plugins>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-basics</artifactId>
-                    <version>0.6.2</version>
-                  </plugin>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-value-constructor</artifactId>
-                    <version>3.0</version>
-                  </plugin>
-                </plugins>
-                <schemaDirectory>src/main/resources/schema</schemaDirectory>
-                <schemaIncludes>
-	          <include>event.xsd</include>
-	        </schemaIncludes>
-                <generatePackage>edu.uci.ics.asterix.event.schema.event</generatePackage>
-                <generateDirectory>${project.build.directory}/generated-sources/event</generateDirectory>
-              </configuration>
-            </execution>
-            <execution>
-              <id>pattern</id>
-              <goals>
-                <goal>generate</goal>
-              </goals>
-              <configuration>
-                <args>
-                  <arg>-Xsetters</arg>
-                  <arg>-Xvalue-constructor</arg>
-                </args>
-                <plugins>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-basics</artifactId>
-                    <version>0.6.2</version>
-                  </plugin>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-value-constructor</artifactId>
-                    <version>3.0</version>
-                  </plugin>
-                </plugins>
-                <schemaDirectory>src/main/resources/schema</schemaDirectory>
-                <schemaIncludes>
-	          <include>pattern.xsd</include>
-	        </schemaIncludes>
-                <generatePackage>edu.uci.ics.asterix.event.schema.pattern</generatePackage>
-                <generateDirectory>${project.build.directory}/generated-sources/pattern</generateDirectory>
-              </configuration>
-            </execution>
-            <execution>
-              <id>cluster</id>
-              <goals>
-                <goal>generate</goal>
-              </goals>
-              <configuration>
-                <args>
-                  <arg>-Xsetters</arg>
-                  <arg>-Xvalue-constructor</arg>
-                </args>
-                <plugins>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-basics</artifactId>
-                    <version>0.6.2</version>
-                  </plugin>
-                  <plugin>
-                    <groupId>org.jvnet.jaxb2_commons</groupId>
-                    <artifactId>jaxb2-value-constructor</artifactId>
-                    <version>3.0</version>
-                  </plugin>
-                </plugins>
-                <schemaDirectory>src/main/resources/schema</schemaDirectory>
-                <schemaIncludes>
-	          <include>cluster.xsd</include>
-	        </schemaIncludes>
-                <generatePackage>edu.uci.ics.asterix.event.schema.cluster</generatePackage>
-                <generateDirectory>${project.build.directory}/generated-sources/cluster</generateDirectory>
-                <bindingDirectory>src/main/resources/schema</bindingDirectory>
-                <bindingIncludes>
-                  <bindingInclude>jaxb-bindings.xjb</bindingInclude>
-                </bindingIncludes>
-              </configuration>
-            </execution>
-          </executions>
-       </plugin>
-       <plugin>
-         <artifactId>maven-assembly-plugin</artifactId>
-         <version>2.2-beta-2</version>
-         <executions>
-           <execution>
-             <configuration>
-               <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
-             </configuration>
-             <phase>package</phase>
-             <goals>
-               <goal>attached</goal>
-             </goals>
-           </execution>
-         </executions>
-       </plugin>
-    </plugins>
-  </build>
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
 
-  <dependencies>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>4.8.1</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>args4j</groupId>
-      <artifactId>args4j</artifactId>
-      <version>2.0.12</version>
-      <type>jar</type>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.jvnet.jaxb2_commons</groupId>
-      <artifactId>jaxb2-value-constructor</artifactId>
-      <version>3.0</version>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-      <version>1.4</version>
-    </dependency>
-   <dependency>
-     <groupId>edu.uci.ics.asterix</groupId>
-     <artifactId>asterix-common</artifactId>
-     <version>0.8.1-SNAPSHOT</version>
-     <type>jar</type>
-     <scope>compile</scope>
-    </dependency>
-   <dependency>
-   	<groupId>log4j</groupId>
-   	<artifactId>log4j</artifactId>
-   	<version>1.2.9</version>
-   </dependency>
-  </dependencies>
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.6</source>
+					<target>1.6</target>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.jvnet.jaxb2.maven2</groupId>
+				<artifactId>maven-jaxb2-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>event</id>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+						<configuration>
+							<args>
+								<arg>-Xsetters</arg>
+								<arg>-Xvalue-constructor</arg>
+							</args>
+							<plugins>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-basics</artifactId>
+									<version>0.6.2</version>
+								</plugin>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-value-constructor</artifactId>
+									<version>3.0</version>
+								</plugin>
+							</plugins>
+							<schemaDirectory>src/main/resources/schema</schemaDirectory>
+							<schemaIncludes>
+								<include>event.xsd</include>
+							</schemaIncludes>
+							<generatePackage>edu.uci.ics.asterix.event.schema.event</generatePackage>
+							<generateDirectory>${project.build.directory}/generated-sources/event</generateDirectory>
+						</configuration>
+					</execution>
+					<execution>
+						<id>configuration</id>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+						<configuration>
+							<args>
+								<arg>-Xsetters</arg>
+								<arg>-Xvalue-constructor</arg>
+							</args>
+							<plugins>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-basics</artifactId>
+									<version>0.6.2</version>
+								</plugin>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-value-constructor</artifactId>
+									<version>3.0</version>
+								</plugin>
+							</plugins>
+							<schemaDirectory>src/main/resources/schema</schemaDirectory>
+							<schemaIncludes>
+								<include>installer-conf.xsd</include>
+							</schemaIncludes>
+							<generatePackage>edu.uci.ics.asterix.installer.schema.conf</generatePackage>
+							<generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
+						</configuration>
+					</execution>
+					<execution>
+						<id>pattern</id>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+						<configuration>
+							<args>
+								<arg>-Xsetters</arg>
+								<arg>-Xvalue-constructor</arg>
+							</args>
+							<plugins>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-basics</artifactId>
+									<version>0.6.2</version>
+								</plugin>
+								<plugin>
+									<groupId>org.jvnet.jaxb2_commons</groupId>
+									<artifactId>jaxb2-value-constructor</artifactId>
+									<version>3.0</version>
+								</plugin>
+							</plugins>
+							<schemaDirectory>src/main/resources/schema</schemaDirectory>
+							<schemaIncludes>
+								<include>pattern.xsd</include>
+							</schemaIncludes>
+							<generatePackage>edu.uci.ics.asterix.event.schema.pattern</generatePackage>
+							<generateDirectory>${project.build.directory}/generated-sources/pattern</generateDirectory>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<version>2.2-beta-2</version>
+				<executions>
+					<execution>
+						<configuration>
+							<descriptor>src/main/assembly/binary-assembly.xml</descriptor>
+						</configuration>
+						<phase>package</phase>
+						<goals>
+							<goal>attached</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+	<dependencies>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.8.1</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>args4j</groupId>
+			<artifactId>args4j</artifactId>
+			<version>2.0.12</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.jvnet.jaxb2_commons</groupId>
+			<artifactId>jaxb2-value-constructor</artifactId>
+			<version>3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>commons-io</groupId>
+			<artifactId>commons-io</artifactId>
+			<version>1.4</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.zookeeper</groupId>
+			<artifactId>zookeeper</artifactId>
+			<version>3.4.5</version>
+			<exclusions>
+				<exclusion>
+					<groupId>com.sun.jmx</groupId>
+					<artifactId>jmxri</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jdmk</groupId>
+					<artifactId>jmxtools</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>javax.jms</groupId>
+					<artifactId>jms</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-common</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>log4j</groupId>
+			<artifactId>log4j</artifactId>
+			<version>1.2.9</version>
+		</dependency>
+	</dependencies>
 </project>
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
index 641b37b..2317cfa 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
@@ -29,7 +29,7 @@
 
 import edu.uci.ics.asterix.event.management.DefaultOutputHandler;
 import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
 import edu.uci.ics.asterix.event.management.IOutputHandler;
 import edu.uci.ics.asterix.event.management.Randomizer;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
@@ -41,7 +41,7 @@
 public class EventDriver {
 
     public static final String CLIENT_NODE_ID = "client_node";
-    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null);
+    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null, null);
 
     private static String eventsDir;
     private static Events events;
@@ -87,15 +87,6 @@
 
     }
 
-    public static EventrixClient getClient(String eventsDir, Cluster cluster, boolean dryRun) throws Exception {
-        return new EventrixClient(eventsDir, cluster, dryRun, new DefaultOutputHandler());
-    }
-
-    public static EventrixClient getClient(String eventsDir, Cluster cluster, boolean dryRun,
-            IOutputHandler outputHandler) throws Exception {
-        return new EventrixClient(eventsDir, cluster, dryRun, outputHandler);
-    }
-
     public static void main(String[] args) throws Exception {
         String eventsHome = System.getenv("EVENT_HOME");
         if (eventsHome == null) {
@@ -119,9 +110,9 @@
             if (!eventConfig.dryRun) {
                 prepare(cluster);
             }
-            EventrixClient client = new EventrixClient(eventsDir, cluster, eventConfig.dryRun,
-                    new DefaultOutputHandler());
-            client.submit(patterns);
+            //AsterixEventServiceClient client = new AsterixEventServiceClient(eventsDir, cluster, eventConfig.dryRun,
+            //      new DefaultOutputHandler());
+            // client.submit(patterns);
             if (!eventConfig.dryRun) {
                 cleanup(cluster);
             }
@@ -156,4 +147,5 @@
         }
         EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "cleanup.sh", args);
     }
+
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
similarity index 74%
rename from asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
index af46e63..83ed5f4 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
@@ -12,12 +12,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.event.xml;
+package edu.uci.ics.asterix.event.error;
 
-public class PatternParser {
+public class EventException extends Exception {
 
-	public static void parsePattern(String path){
-		
-	}
+    private static final long serialVersionUID = 1L;
+
+    public EventException(String message) {
+        super(message);
+    }
+
 }
-
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java
new file mode 100644
index 0000000..3bc795a
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2012 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.event.error;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import edu.uci.ics.asterix.event.management.IOutputHandler;
+import edu.uci.ics.asterix.event.management.OutputAnalysis;
+import edu.uci.ics.asterix.event.model.EventList.EventType;
+import edu.uci.ics.asterix.event.schema.pattern.Event;
+
+public class OutputHandler implements IOutputHandler {
+
+    public static IOutputHandler INSTANCE = new OutputHandler();
+
+    private OutputHandler() {
+
+    }
+
+    public OutputAnalysis reportEventOutput(Event event, String output) {
+
+        EventType eventType = EventType.valueOf(event.getType().toUpperCase());
+        boolean ignore = true;
+        String trimmedOutput = output.trim();
+        StringBuffer errorMessage = new StringBuffer();
+        switch (eventType) {
+            case FILE_TRANSFER:
+                if (trimmedOutput.length() > 0) {
+                    if (output.contains("Permission denied") || output.contains("cannot find or open")) {
+                        ignore = false;
+                        break;
+                    }
+                }
+                break;
+
+            case BACKUP:
+            case RESTORE:
+                if (trimmedOutput.length() > 0) {
+                    if (trimmedOutput.contains("AccessControlException")) {
+                        errorMessage.append("Insufficient permissions on back up directory");
+                        ignore = false;
+                    }
+                    if (output.contains("does not exist") || output.contains("File exist")
+                            || (output.contains("No such file or directory"))) {
+                        ignore = true;
+                    } else {
+                        ignore = false;
+                    }
+                }
+                break;
+
+            case NODE_INFO:
+                Properties p = new Properties();
+                try {
+                    p.load(new ByteArrayInputStream(trimmedOutput.getBytes()));
+                } catch (IOException e) {
+                }
+                String javaVersion = (String) p.get("java_version");
+                if (p.get("java_version") == null) {
+                    errorMessage.append("Java not installed on " + event.getNodeid().getValue().getAbsvalue());
+                    ignore = false;
+                } else if (!javaVersion.contains("1.7")) {
+                    errorMessage.append("Asterix requires Java 1.7.x. Incompatible version found on  "
+                            + event.getNodeid().getValue().getAbsvalue() + "\n");
+                    ignore = false;
+                }
+                break;
+        }
+        if (ignore) {
+            return new OutputAnalysis(true, null);
+        } else {
+            return new OutputAnalysis(false, errorMessage.toString());
+        }
+    }
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
similarity index 78%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
index 5be3f89..b329171 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2013 by The Regents of the University of California
+ * Copyright 2009-2012 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
@@ -12,30 +12,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.error;
+package edu.uci.ics.asterix.event.error;
 
 import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.model.ProcessInfo;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
 
 public class VerificationUtil {
 
-	private static final String VERIFY_SCRIPT_PATH = InstallerDriver
-			.getManagixHome()
-			+ File.separator
-			+ InstallerDriver.MANAGIX_INTERNAL_DIR
+	private static final String VERIFY_SCRIPT_PATH = AsterixEventService
+			.getEventHome()
 			+ File.separator
 			+ "scripts"
-			+ File.separator + "verify.sh";
+			+ File.separator
+			+ "verify.sh";
 
 	public static AsterixRuntimeState getAsterixRuntimeState(
 			AsterixInstance instance) throws Exception {
@@ -49,8 +48,8 @@
 			args.add(instance.getName() + "_" + node.getId());
 		}
 		Thread.sleep(2000);
-		String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH,
-				args);
+		String output = AsterixEventServiceUtil.executeLocalScript(
+				VERIFY_SCRIPT_PATH, args);
 		boolean ccRunning = true;
 		List<String> failedNCs = new ArrayList<String>();
 		String[] infoFields;
@@ -97,10 +96,10 @@
 				for (String failedNC : state.getFailedNCs()) {
 					summary.append(failedNC + "\n");
 				}
-				instance.setState(State.UNUSABLE);
+				// instance.setState(State.UNUSABLE);
 			}
 			if (!(instance.getState().equals(State.UNUSABLE))) {
-				instance.setState(State.ACTIVE);
+				 instance.setState(State.ACTIVE);
 			}
 		} else {
 			if (state.getProcesses() != null && state.getProcesses().size() > 0) {
@@ -108,9 +107,9 @@
 				for (ProcessInfo pInfo : state.getProcesses()) {
 					summary.append(pInfo + "\n");
 				}
-				instance.setState(State.UNUSABLE);
+				// instance.setState(State.UNUSABLE);
 			} else {
-				instance.setState(State.INACTIVE);
+				// instance.setState(State.INACTIVE);
 			}
 		}
 		state.setSummary(summary.toString());
@@ -130,9 +129,7 @@
 			errorCheck.append("\n HDFS backup directory not configured");
 		}
 		if (errorCheck.length() > 0) {
-			throw new Exception("Incomplete hdfs configuration in "
-					+ InstallerDriver.getManagixHome() + File.separator
-					+ InstallerDriver.MANAGIX_CONF_XML + errorCheck);
+			throw new Exception("Incomplete hdfs configuration" + errorCheck);
 		}
 	}
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
similarity index 64%
rename from asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
index 03f4061..8947247 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
@@ -16,6 +16,7 @@
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -24,6 +25,7 @@
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
 import edu.uci.ics.asterix.event.driver.EventDriver;
@@ -35,10 +37,12 @@
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.event.schema.pattern.Value;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 
-public class EventrixClient {
+public class AsterixEventServiceClient {
 
-    private static final Logger LOGGER = Logger.getLogger(EventrixClient.class.getName());
+    private static final Logger LOGGER = Logger.getLogger(AsterixEventServiceClient.class.getName());
 
     private EventTask[] tasks;
     private boolean dryRun = false;
@@ -48,21 +52,26 @@
     private IPatternListener listener;
     private IOutputHandler outputHandler;
     private Events events;
-    private String eventsDir;
+    private String eventsHomeDir;
+    private Configuration configuration;
 
-    public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun, IOutputHandler outputHandler)
-            throws Exception {
-        this.eventsDir = eventsDir;
+    public AsterixEventServiceClient(Configuration configuration, String eventsHomeDir, Cluster cluster,
+            boolean transferArtifacts, boolean dryRun, IOutputHandler outputHandler) throws Exception {
+        this.eventsHomeDir = eventsHomeDir;
         this.events = initializeEvents();
         this.cluster = cluster;
         this.dryRun = dryRun;
+        this.configuration = configuration;
         this.outputHandler = outputHandler;
-        if (!dryRun) {
-            initializeCluster(eventsDir);
+        if (!dryRun && transferArtifacts) {
+            initializeCluster(getEventsDir());
         }
     }
 
     public void submit(Patterns patterns) throws Exception {
+        if (patterns.getPattern().isEmpty()) {
+            return;
+        }
         initTasks(patterns);
         try {
             waitForCompletion();
@@ -103,7 +112,7 @@
     }
 
     public String getEventsDir() {
-        return eventsDir;
+        return eventsHomeDir + File.separator + AsterixEventServiceUtil.EVENT_DIR;
     }
 
     public synchronized void notifyCompletion(EventTaskReport report) {
@@ -156,25 +165,36 @@
         submit(patterns);
     }
 
-	private Patterns initPattern(String eventsDir) {
-		Nodeid nodeid = new Nodeid(new Value(null,
-				EventDriver.CLIENT_NODE.getId()));
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		String workingDir = cluster.getWorkingDir().getDir();
-		String username = cluster.getUsername() == null ? System
-				.getProperty("user.name") : cluster.getUsername();
-		patternList.add(getDirectoryTransferPattern(username, eventsDir,
-				nodeid, cluster.getMasterNode().getClusterIp(), workingDir));
+    private Patterns initPattern(String eventsDir) throws Exception {
+        Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        String workingDir = cluster.getWorkingDir().getDir();
+        String username = cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername();
+        patternList.add(getDirectoryTransferPattern(username, eventsDir, nodeid,
+                cluster.getMasterNode().getClusterIp(), workingDir));
 
-		if (!cluster.getWorkingDir().isNFS()) {
-			for (Node node : cluster.getNode()) {
-				patternList.add(getDirectoryTransferPattern(username,
-						eventsDir, nodeid, node.getClusterIp(), workingDir));
-			}
-		}
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
+        Marshaller marshaller = ctx.createMarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        String outputPathDir = System.getProperty("java.io.tmpdir") + File.separator + "conf-"
+                + System.getProperty("user.name");
+        new File(outputPathDir).mkdirs();
+        String outputPath = outputPathDir + File.separator + "configuration.xml";
+        marshaller.marshal(configuration, new FileOutputStream(outputPath));
+
+        patternList.add(getFileTransferPattern(username, outputPath, nodeid, cluster.getMasterNode().getClusterIp(),
+                workingDir));
+
+        if (!cluster.getWorkingDir().isNFS()) {
+            for (Node node : cluster.getNode()) {
+                patternList.add(getDirectoryTransferPattern(username, eventsDir, nodeid, node.getClusterIp(),
+                        workingDir));
+
+            }
+        }
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
     private Pattern getDirectoryTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
             String destDir) {
@@ -183,16 +203,27 @@
         return new Pattern(null, 1, null, event);
     }
 
+    private Pattern getFileTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
+            String destDir) {
+        String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
+        Event event = new Event("file_transfer", srcNode, pargs);
+        return new Pattern(null, 1, null, event);
+    }
+
     public IOutputHandler getErrorHandler() {
         return outputHandler;
     }
 
     private Events initializeEvents() throws JAXBException, FileNotFoundException {
-        File file = new File(eventsDir + File.separator + "events" + File.separator + "events.xml");
+        File file = new File(getEventsDir() + File.separator + "events.xml");
         JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
         Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
         events = (Events) unmarshaller.unmarshal(file);
         return events;
     }
 
+    public String getEventsHomeDir() {
+        return eventsHomeDir;
+    }
+
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
index d05c5ab..f7155f3 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.asterix.event.schema.pattern.Event;
 
+
 public class DefaultOutputHandler implements IOutputHandler {
 
     @Override
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index 45499f4..82b6e9e 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -23,11 +23,13 @@
 
 import org.apache.commons.io.IOUtils;
 
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.event.driver.EventDriver;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
 import edu.uci.ics.asterix.event.schema.cluster.Property;
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
 
 public class EventExecutor {
 
@@ -40,10 +42,11 @@
     private static final String DAEMON = "DAEMON";
 
     public void executeEvent(Node node, String script, List<String> args, boolean isDaemon, Cluster cluster,
-            Pattern pattern, IOutputHandler outputHandler, EventrixClient client) throws IOException {
+            Pattern pattern, IOutputHandler outputHandler, AsterixEventServiceClient client) throws IOException {
         List<String> pargs = new ArrayList<String>();
         pargs.add("/bin/bash");
-        pargs.add(client.getEventsDir() + File.separator + "scripts" + File.separator + EXECUTE_SCRIPT);
+        pargs.add(client.getEventsHomeDir() + File.separator + AsterixEventServiceUtil.EVENT_DIR + File.separator
+                + EXECUTE_SCRIPT);
         StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getClusterIp() + " ");
         boolean isMasterNode = node.getId().equals(cluster.getMasterNode().getId());
 
@@ -60,6 +63,13 @@
                         if (javaOpts != null) {
                             builder.append(javaOpts);
                         }
+                        if (node.getDebugPort() != null) {
+                            int debugPort = node.getDebugPort().intValue();
+                            if (!javaOpts.contains("-Xdebug")) {
+                                builder.append((" "
+                                        + "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=" + debugPort));
+                            }
+                        }
                         builder.append("\"");
                         envBuffer.append("JAVA_OPTS" + "=" + builder + " ");
                     }
@@ -71,6 +81,13 @@
                         if (javaOpts != null) {
                             builder.append(javaOpts);
                         }
+                        if (node.getDebugPort() != null) {
+                            int debugPort = node.getDebugPort().intValue();
+                            if (!javaOpts.contains("-Xdebug")) {
+                                builder.append((" "
+                                        + "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=" + debugPort));
+                            }
+                        }
                         builder.append("\"");
                         envBuffer.append("JAVA_OPTS" + "=" + builder + " ");
                     }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
index a764f9a..f98230b 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
@@ -20,6 +20,7 @@
 import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
+
 import org.apache.log4j.Logger;
 
 import edu.uci.ics.asterix.event.driver.EventDriver;
@@ -30,134 +31,124 @@
 
 public class EventTask extends TimerTask {
 
-	public static enum State {
-		INITIALIZED, IN_PROGRESS, COMPLETED, FAILED
-	}
+    public static enum State {
+        INITIALIZED,
+        IN_PROGRESS,
+        COMPLETED,
+        FAILED
+    }
 
-	private static final Logger logger = Logger.getLogger(EventTask.class
-			.getName());
+    private static final Logger logger = Logger.getLogger(EventTask.class.getName());
 
-	private Pattern pattern;
-	private Event event;
-	private long interval = 0;
-	private long initialDelay = 0;
-	private int maxOccurs = Integer.MAX_VALUE;
-	private int occurrenceCount = 0;
-	private Timer timer;
-	private String taskScript;
-	private Node location;
-	private List<String> taskArgs;
-	private EventrixClient client;
-	private List<Node> candidateLocations;
-	private boolean dynamicLocation = false;
-	private boolean reuseLocation = false;
-	private State state;
+    private Pattern pattern;
+    private Event event;
+    private long interval = 0;
+    private long initialDelay = 0;
+    private int maxOccurs = Integer.MAX_VALUE;
+    private int occurrenceCount = 0;
+    private Timer timer;
+    private String taskScript;
+    private Node location;
+    private List<String> taskArgs;
+    private AsterixEventServiceClient client;
+    private List<Node> candidateLocations;
+    private boolean dynamicLocation = false;
+    private boolean reuseLocation = false;
+    private State state;
 
-	
+    public EventTask(Pattern pattern, AsterixEventServiceClient client) {
+        this.pattern = pattern;
+        this.client = client;
+        Period period = pattern.getPeriod();
+        if (period != null && period.getAbsvalue() != null) {
+            this.interval = EventUtil.parseTimeInterval(period.getAbsvalue(), period.getUnit());
+        }
+        if (pattern.getDelay() != null) {
+            this.initialDelay = EventUtil.parseTimeInterval(new ValueType(pattern.getDelay().getValue()), pattern
+                    .getDelay().getUnit());
+        }
+        if (pattern.getMaxOccurs() != null) {
+            this.maxOccurs = pattern.getMaxOccurs();
+        }
+        this.timer = new Timer();
+        taskArgs = EventUtil.getEventArgs(pattern);
+        candidateLocations = EventUtil.getCandidateLocations(pattern, client.getCluster());
+        if (pattern.getEvent().getNodeid().getValue().getRandom() != null && period != null && maxOccurs > 1) {
+            dynamicLocation = true;
+            reuseLocation = pattern.getEvent().getNodeid().getValue().getRandom().getRange().isReuse();
+        } else {
+            location = EventUtil.getEventLocation(pattern, candidateLocations, client.getCluster());
+        }
+        String scriptsDir;
+        if (location.getId().equals(EventDriver.CLIENT_NODE_ID)) {
+            scriptsDir = client.getEventsDir();
+        } else {
+            scriptsDir = client.getCluster().getWorkingDir().getDir() + File.separator + "events";
+        }
+        event = EventUtil.getEvent(pattern, client.getEvents());
+        taskScript = scriptsDir + File.separator + event.getScript();
+        state = State.INITIALIZED;
+    }
 
-	public EventTask(Pattern pattern, EventrixClient client) {
-		this.pattern = pattern;
-		this.client = client;
-		Period period = pattern.getPeriod();
-		if (period != null && period.getAbsvalue() != null) {
-			this.interval = EventUtil.parseTimeInterval(period.getAbsvalue(),
-					period.getUnit());
-		}
-		if (pattern.getDelay() != null) {
-			this.initialDelay = EventUtil.parseTimeInterval(new ValueType(
-					pattern.getDelay().getValue()), pattern.getDelay()
-					.getUnit());
-		}
-		if (pattern.getMaxOccurs() != null) {
-			this.maxOccurs = pattern.getMaxOccurs();
-		}
-		this.timer = new Timer();
-		taskArgs = EventUtil.getEventArgs(pattern);
-		candidateLocations = EventUtil.getCandidateLocations(pattern,
-				client.getCluster());
-		if (pattern.getEvent().getNodeid().getValue().getRandom() != null
-				&& period != null && maxOccurs > 1) {
-			dynamicLocation = true;
-			reuseLocation = pattern.getEvent().getNodeid().getValue()
-					.getRandom().getRange().isReuse();
-		} else {
-			location = EventUtil.getEventLocation(pattern, candidateLocations,
-					client.getCluster());
-		}
-		String scriptsDir;
-		if (location.getId().equals(EventDriver.CLIENT_NODE_ID)) {
-			scriptsDir = client.getEventsDir() + File.separator + "events";
-		} else {
-			scriptsDir = client.getCluster().getWorkingDir().getDir()
-					+ File.separator + "eventrix" + File.separator + "events";
-		}
-		event = EventUtil.getEvent(pattern, client.getEvents());
-		taskScript = scriptsDir + File.separator + event.getScript();
-		state = State.INITIALIZED;
-	}
+    public void start() {
+        if (interval > 0) {
+            timer.schedule(this, initialDelay, interval);
+        } else {
+            timer.schedule(this, initialDelay);
+        }
+    }
 
-	public void start() {
-		if (interval > 0) {
-			timer.schedule(this, initialDelay, interval);
-		} else {
-			timer.schedule(this, initialDelay);
-		}
-	}
+    @Override
+    public void run() {
+        if (candidateLocations.size() == 0) {
+            timer.cancel();
+            client.notifyCompletion(new EventTaskReport(this));
+        } else {
+            if (dynamicLocation) {
+                location = EventUtil.getEventLocation(pattern, candidateLocations, client.getCluster());
+                if (!reuseLocation) {
+                    candidateLocations.remove(location);
+                }
+            }
 
-	@Override
-	public void run() {
-		if (candidateLocations.size() == 0) {
-			timer.cancel();
-			client.notifyCompletion(new EventTaskReport(this));
-		} else {
-			if (dynamicLocation) {
-				location = EventUtil.getEventLocation(pattern,
-						candidateLocations, client.getCluster());
-				if (!reuseLocation) {
-					candidateLocations.remove(location);
-				}
-			}
+            logger.debug(EventUtil.dateFormat.format(new Date()) + " " + "EVENT "
+                    + pattern.getEvent().getType().toUpperCase() + " at " + location.getId().toUpperCase());
+            try {
+                if (!client.isDryRun()) {
+                    new EventExecutor().executeEvent(location, taskScript, taskArgs, event.isDaemon(),
+                            client.getCluster(), pattern, client.getErrorHandler(), client);
+                }
+                occurrenceCount++;
+                if (occurrenceCount >= maxOccurs) {
+                    timer.cancel();
+                    client.notifyCompletion(new EventTaskReport(this));
+                }
+            } catch (IOException ioe) {
+                timer.cancel();
+                client.notifyCompletion(new EventTaskReport(this, false, ioe));
+            }
+        }
 
-			logger.debug(EventUtil.dateFormat.format(new Date()) + " "
-					+ "EVENT " + pattern.getEvent().getType().toUpperCase()
-					+ " at " + location.getId().toUpperCase());
-			try {
-				if (!client.isDryRun()) {
-					new EventExecutor().executeEvent(location, taskScript,
-							taskArgs, event.isDaemon(), client.getCluster(),
-							pattern, client.getErrorHandler(), client);
-				}
-				occurrenceCount++;
-				if (occurrenceCount >= maxOccurs) {
-					timer.cancel();
-					client.notifyCompletion(new EventTaskReport(this));
-				}
-			} catch (IOException ioe) {
-				timer.cancel();
-				client.notifyCompletion(new EventTaskReport(this, false, ioe));
-			}
-		}
+    }
 
-	}
+    public Node getLocation() {
+        return location;
+    }
 
-	public Node getLocation() {
-		return location;
-	}
+    public long getInterval() {
+        return interval;
+    }
 
-	public long getInterval() {
-		return interval;
-	}
+    public long getInitialDelay() {
+        return initialDelay;
+    }
 
-	public long getInitialDelay() {
-		return initialDelay;
-	}
+    public Pattern getPattern() {
+        return pattern;
+    }
 
-	public Pattern getPattern() {
-		return pattern;
-	}
-
-	public State getState() {
-		return state;
-	}
+    public State getState() {
+        return state;
+    }
 
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
index 887e272..49b7abf 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
@@ -16,7 +16,6 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.math.BigInteger;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -36,252 +35,232 @@
 
 public class EventUtil {
 
-	public static final String EVENTS_DIR = "events";
-	public static final String CLUSTER_CONF = "config/cluster.xml";
-	public static final String PATTERN_CONF = "config/pattern.xml";
-	public static final DateFormat dateFormat = new SimpleDateFormat(
-			"yyyy/MM/dd HH:mm:ss");
-	public static final String NC_JAVA_OPTS = "nc.java.opts";
-	public static final String CC_JAVA_OPTS = "cc.java.opts";
+    public static final String EVENTS_DIR = "events";
+    public static final String CLUSTER_CONF = "config/cluster.xml";
+    public static final String PATTERN_CONF = "config/pattern.xml";
+    public static final DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+    public static final String NC_JAVA_OPTS = "nc.java.opts";
+    public static final String CC_JAVA_OPTS = "cc.java.opts";
 
-	private static final String IP_LOCATION = "IP_LOCATION";
-	private static final String CLUSTER_ENV = "ENV";
-	private static final String SCRIPT = "SCRIPT";
-	private static final String ARGS = "ARGS";
-	private static final String EXECUTE_SCRIPT = "events/execute.sh";
-	private static final String LOCALHOST = "localhost";
-	private static final String LOCALHOST_IP = "127.0.0.1";
+    private static final String IP_LOCATION = "IP_LOCATION";
+    private static final String CLUSTER_ENV = "ENV";
+    private static final String SCRIPT = "SCRIPT";
+    private static final String ARGS = "ARGS";
+    private static final String EXECUTE_SCRIPT = "events/execute.sh";
+    private static final String LOCALHOST = "localhost";
+    private static final String LOCALHOST_IP = "127.0.0.1";
 
-	public static Cluster getCluster(String clusterConfigurationPath)
-			throws JAXBException {
-		File file = new File(clusterConfigurationPath);
-		JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
-		Unmarshaller unmarshaller = ctx.createUnmarshaller();
-		Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
-		if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
-			cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
-		}
-		for (Node node : cluster.getNode()) {
-			if (node.getClusterIp().equals(LOCALHOST)) {
-				node.setClusterIp(LOCALHOST_IP);
-			}
-		}
-		return cluster;
-	}
+    public static Cluster getCluster(String clusterConfigurationPath) throws JAXBException {
+        File file = new File(clusterConfigurationPath);
+        JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+        Unmarshaller unmarshaller = ctx.createUnmarshaller();
+        Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
+        if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
+            cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
+        }
+        for (Node node : cluster.getNode()) {
+            if (node.getClusterIp().equals(LOCALHOST)) {
+                node.setClusterIp(LOCALHOST_IP);
+            }
+        }
+        return cluster;
+    }
 
-	public static long parseTimeInterval(ValueType v, String unit)
-			throws IllegalArgumentException {
-		int val = 0;
-		switch (v.getType()) {
-		case ABS:
-			val = Integer.parseInt(v.getAbsoluteValue());
-			break;
-		case RANDOM_MIN_MAX:
-			val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
-			break;
-		case RANDOM_RANGE:
-			String[] values = v.getRangeSet();
-			val = Integer.parseInt(values[Randomizer.getInstance()
-					.getRandomInt(0, values.length - 1)]);
-			break;
-		}
-		return computeInterval(val, unit);
-	}
+    public static long parseTimeInterval(ValueType v, String unit) throws IllegalArgumentException {
+        int val = 0;
+        switch (v.getType()) {
+            case ABS:
+                val = Integer.parseInt(v.getAbsoluteValue());
+                break;
+            case RANDOM_MIN_MAX:
+                val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
+                break;
+            case RANDOM_RANGE:
+                String[] values = v.getRangeSet();
+                val = Integer.parseInt(values[Randomizer.getInstance().getRandomInt(0, values.length - 1)]);
+                break;
+        }
+        return computeInterval(val, unit);
+    }
 
-	public static long parseTimeInterval(String v, String unit)
-			throws IllegalArgumentException {
-		int value = Integer.parseInt(v);
-		return computeInterval(value, unit);
-	}
+    public static long parseTimeInterval(String v, String unit) throws IllegalArgumentException {
+        int value = Integer.parseInt(v);
+        return computeInterval(value, unit);
+    }
 
-	private static long computeInterval(int val, String unit) {
-		int vmult = 1;
-		if ("hr".equalsIgnoreCase(unit)) {
-			vmult = 3600 * 1000;
-		} else if ("min".equalsIgnoreCase(unit)) {
-			vmult = 60 * 1000;
-		} else if ("sec".equalsIgnoreCase(unit)) {
-			vmult = 1000;
-		} else
-			throw new IllegalArgumentException(
-					" invalid unit value specified for frequency (hr,min,sec)");
-		return val * vmult;
+    private static long computeInterval(int val, String unit) {
+        int vmult = 1;
+        if ("hr".equalsIgnoreCase(unit)) {
+            vmult = 3600 * 1000;
+        } else if ("min".equalsIgnoreCase(unit)) {
+            vmult = 60 * 1000;
+        } else if ("sec".equalsIgnoreCase(unit)) {
+            vmult = 1000;
+        } else
+            throw new IllegalArgumentException(" invalid unit value specified for frequency (hr,min,sec)");
+        return val * vmult;
 
-	}
+    }
 
-	public static Event getEvent(Pattern pattern, Events events) {
-		for (Event event : events.getEvent()) {
-			if (event.getType().equals(pattern.getEvent().getType())) {
-				return event;
-			}
-		}
-		throw new IllegalArgumentException(" Unknown event type"
-				+ pattern.getEvent().getType());
-	}
+    public static Event getEvent(Pattern pattern, Events events) {
+        for (Event event : events.getEvent()) {
+            if (event.getType().equals(pattern.getEvent().getType())) {
+                return event;
+            }
+        }
+        throw new IllegalArgumentException(" Unknown event type" + pattern.getEvent().getType());
+    }
 
-	public static Node getEventLocation(Pattern pattern,
-			List<Node> candidateLocations, Cluster cluster) {
-		ValueType value = new ValueType(pattern.getEvent().getNodeid()
-				.getValue());
-		Node location = null;
-		Type vtype = value.getType();
+    public static Node getEventLocation(Pattern pattern, List<Node> candidateLocations, Cluster cluster) {
+        ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
+        Node location = null;
+        Type vtype = value.getType();
 
-		switch (vtype) {
-		case ABS:
-			location = getNodeFromId(value.getAbsoluteValue(), cluster);
-			break;
-		case RANDOM_RANGE:
-			int nodeIndex = Randomizer.getInstance().getRandomInt(0,
-					candidateLocations.size() - 1);
-			location = candidateLocations.get(nodeIndex);
-			break;
-		case RANDOM_MIN_MAX:
-			throw new IllegalStateException(
-					" Canont configure a min max value range for location");
-		}
-		return location;
+        switch (vtype) {
+            case ABS:
+                location = getNodeFromId(value.getAbsoluteValue(), cluster);
+                break;
+            case RANDOM_RANGE:
+                int nodeIndex = Randomizer.getInstance().getRandomInt(0, candidateLocations.size() - 1);
+                location = candidateLocations.get(nodeIndex);
+                break;
+            case RANDOM_MIN_MAX:
+                throw new IllegalStateException(" Canont configure a min max value range for location");
+        }
+        return location;
 
-	}
+    }
 
-	public static List<Node> getCandidateLocations(Pattern pattern,
-			Cluster cluster) {
-		ValueType value = new ValueType(pattern.getEvent().getNodeid()
-				.getValue());
-		List<Node> candidateList = new ArrayList<Node>();
-		switch (value.getType()) {
-		case ABS:
-			candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
-			break;
-		case RANDOM_RANGE:
-			boolean anyOption = false;
-			String[] values = value.getRangeSet();
-			for (String v : values) {
-				if (v.equalsIgnoreCase("ANY")) {
-					anyOption = true;
-				}
-			}
-			if (anyOption) {
-				for (Node node : cluster.getNode()) {
-					candidateList.add(node);
-				}
-			} else {
-				boolean found = false;
-				for (String v : values) {
-					for (Node node : cluster.getNode()) {
-						if (node.getId().equals(v)) {
-							candidateList.add(node);
-							found = true;
-							break;
-						}
-					}
-					if (!found) {
-						throw new IllegalStateException("Unknonw nodeId : " + v);
-					}
-					found = false;
-				}
+    public static List<Node> getCandidateLocations(Pattern pattern, Cluster cluster) {
+        ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
+        List<Node> candidateList = new ArrayList<Node>();
+        switch (value.getType()) {
+            case ABS:
+                candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
+                break;
+            case RANDOM_RANGE:
+                boolean anyOption = false;
+                String[] values = value.getRangeSet();
+                for (String v : values) {
+                    if (v.equalsIgnoreCase("ANY")) {
+                        anyOption = true;
+                    }
+                }
+                if (anyOption) {
+                    for (Node node : cluster.getNode()) {
+                        candidateList.add(node);
+                    }
+                } else {
+                    boolean found = false;
+                    for (String v : values) {
+                        for (Node node : cluster.getNode()) {
+                            if (node.getId().equals(v)) {
+                                candidateList.add(node);
+                                found = true;
+                                break;
+                            }
+                        }
+                        if (!found) {
+                            throw new IllegalStateException("Unknonw nodeId : " + v);
+                        }
+                        found = false;
+                    }
 
-			}
-			String[] excluded = value.getRangeExcluded();
-			if (excluded != null && excluded.length > 0) {
-				List<Node> markedForRemoval = new ArrayList<Node>();
-				for (String exclusion : excluded) {
-					for (Node node : candidateList) {
-						if (node.getId().equals(exclusion)) {
-							markedForRemoval.add(node);
-						}
-					}
-				}
-				candidateList.removeAll(markedForRemoval);
-			}
-			break;
-		case RANDOM_MIN_MAX:
-			throw new IllegalStateException(
-					" Invalid value configured for location");
-		}
-		return candidateList;
-	}
+                }
+                String[] excluded = value.getRangeExcluded();
+                if (excluded != null && excluded.length > 0) {
+                    List<Node> markedForRemoval = new ArrayList<Node>();
+                    for (String exclusion : excluded) {
+                        for (Node node : candidateList) {
+                            if (node.getId().equals(exclusion)) {
+                                markedForRemoval.add(node);
+                            }
+                        }
+                    }
+                    candidateList.removeAll(markedForRemoval);
+                }
+                break;
+            case RANDOM_MIN_MAX:
+                throw new IllegalStateException(" Invalid value configured for location");
+        }
+        return candidateList;
+    }
 
-	private static Node getNodeFromId(String nodeid, Cluster cluster) {
-		if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
-			return EventDriver.CLIENT_NODE;
-		}
+    private static Node getNodeFromId(String nodeid, Cluster cluster) {
+        if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
+            return EventDriver.CLIENT_NODE;
+        }
 
-		if (nodeid.equals(cluster.getMasterNode().getId())) {
-			String logDir = cluster.getMasterNode().getLogDir() == null ? cluster
-					.getLogDir()
-					: cluster.getMasterNode().getLogDir();
-			String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster
-					.getJavaHome()
-					: cluster.getMasterNode().getJavaHome();
-			return new Node(cluster.getMasterNode().getId(), cluster
-					.getMasterNode().getClusterIp(), javaHome, logDir, null,
-					null, null);
-		}
+        if (nodeid.equals(cluster.getMasterNode().getId())) {
+            String logDir = cluster.getMasterNode().getLogDir() == null ? cluster.getLogDir() : cluster.getMasterNode()
+                    .getLogDir();
+            String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster.getJavaHome() : cluster
+                    .getMasterNode().getJavaHome();
+            return new Node(cluster.getMasterNode().getId(), cluster.getMasterNode().getClusterIp(), javaHome, logDir,
+                    null, null, null, cluster.getMasterNode().getDebugPort());
+        }
 
-		List<Node> nodeList = cluster.getNode();
-		for (Node node : nodeList) {
-			if (node.getId().equals(nodeid)) {
-				return node;
-			}
-		}
-		StringBuffer buffer = new StringBuffer();
-		buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
-		buffer.append(cluster.getMasterNode().getId() + ",");
-		for (Node v : cluster.getNode()) {
-			buffer.append(v.getId() + ",");
-		}
-		buffer.deleteCharAt(buffer.length() - 1);
-		throw new IllegalArgumentException("Unknown node id :" + nodeid
-				+ " valid ids:" + buffer);
-	}
+        List<Node> nodeList = cluster.getNode();
+        for (Node node : nodeList) {
+            if (node.getId().equals(nodeid)) {
+                return node;
+            }
+        }
+        StringBuffer buffer = new StringBuffer();
+        buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
+        buffer.append(cluster.getMasterNode().getId() + ",");
+        for (Node v : cluster.getNode()) {
+            buffer.append(v.getId() + ",");
+        }
+        buffer.deleteCharAt(buffer.length() - 1);
+        throw new IllegalArgumentException("Unknown node id :" + nodeid + " valid ids:" + buffer);
+    }
 
-	public static void executeEventScript(Node node, String script,
-			List<String> args, Cluster cluster) throws IOException,
-			InterruptedException {
-		List<String> pargs = new ArrayList<String>();
-		pargs.add("/bin/bash");
-		pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
-		StringBuffer argBuffer = new StringBuffer();
-		String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION
-				+ "=" + node.getClusterIp();
-		if (args != null) {
-			for (String arg : args) {
-				argBuffer.append(arg + " ");
-			}
-		}
-		ProcessBuilder pb = new ProcessBuilder(pargs);
-		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getClusterIp());
-		pb.environment().put(CLUSTER_ENV, env);
-		pb.environment().put(SCRIPT, script);
-		pb.environment().put(ARGS, argBuffer.toString());
-		pb.start();
-	}
+    public static void executeEventScript(Node node, String script, List<String> args, Cluster cluster)
+            throws IOException, InterruptedException {
+        List<String> pargs = new ArrayList<String>();
+        pargs.add("/bin/bash");
+        pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
+        StringBuffer argBuffer = new StringBuffer();
+        String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION + "=" + node.getClusterIp();
+        if (args != null) {
+            for (String arg : args) {
+                argBuffer.append(arg + " ");
+            }
+        }
+        ProcessBuilder pb = new ProcessBuilder(pargs);
+        pb.environment().putAll(EventDriver.getEnvironment());
+        pb.environment().put(IP_LOCATION, node.getClusterIp());
+        pb.environment().put(CLUSTER_ENV, env);
+        pb.environment().put(SCRIPT, script);
+        pb.environment().put(ARGS, argBuffer.toString());
+        pb.start();
+    }
 
-	public static void executeLocalScript(Node node, String script,
-			List<String> args) throws IOException, InterruptedException {
-		List<String> pargs = new ArrayList<String>();
-		pargs.add("/bin/bash");
-		pargs.add(script);
-		if (args != null) {
-			pargs.addAll(args);
-		}
-		ProcessBuilder pb = new ProcessBuilder(pargs);
-		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getClusterIp());
-		pb.start();
-	}
+    public static void executeLocalScript(Node node, String script, List<String> args) throws IOException,
+            InterruptedException {
+        List<String> pargs = new ArrayList<String>();
+        pargs.add("/bin/bash");
+        pargs.add(script);
+        if (args != null) {
+            pargs.addAll(args);
+        }
+        ProcessBuilder pb = new ProcessBuilder(pargs);
+        pb.environment().putAll(EventDriver.getEnvironment());
+        pb.environment().put(IP_LOCATION, node.getClusterIp());
+        pb.start();
+    }
 
-	public static List<String> getEventArgs(Pattern pattern) {
-		List<String> pargs = new ArrayList<String>();
-		if (pattern.getEvent().getPargs() == null) {
-			return pargs;
-		}
-		String[] args = pattern.getEvent().getPargs().split(" ");
-		for (String arg : args) {
-			pargs.add(arg.trim());
-		}
-		return pargs;
-	}
+    public static List<String> getEventArgs(Pattern pattern) {
+        List<String> pargs = new ArrayList<String>();
+        if (pattern.getEvent().getPargs() == null) {
+            return pargs;
+        }
+        String[] args = pattern.getEvent().getPargs().split(" ");
+        for (String arg : args) {
+            pargs.add(arg.trim());
+        }
+        return pargs;
+    }
 
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
index f6cf905..5dd370c 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.asterix.event.schema.pattern.Event;
 
+
 public interface IOutputHandler {
 
     public OutputAnalysis reportEventOutput(Event event, String output);
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
index b619002..2524d9a 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.asterix.event.schema.pattern.Value;
 
+
 public class ValueType {
 
 	public static enum Type {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
similarity index 98%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
index c9a4743..87f14b0 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -30,7 +30,7 @@
     private static final long serialVersionUID = 1L;
 
     private static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
-    
+
     public enum State {
         ACTIVE,
         INACTIVE,
@@ -61,7 +61,6 @@
         this.asterixVersion = asterixVersion;
         this.createdTimestamp = new Date();
         this.backupInfo = new ArrayList<BackupInfo>();
-
     }
 
     public Date getModifiedTimestamp() {
@@ -212,4 +211,5 @@
     public void setAsterixConfiguration(AsterixConfiguration asterixConfiguration) {
         this.asterixConfiguration = asterixConfiguration;
     }
+
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
similarity index 93%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
index e1a5167..b8fbabb 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
@@ -12,13 +12,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
 
 import java.io.Serializable;
 import java.util.List;
 
 public class AsterixRuntimeState implements Serializable {
 
+    private static final long serialVersionUID = 1L;
     private final List<ProcessInfo> processes;
     private final List<String> failedNCs;
     private final boolean ccRunning;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
similarity index 97%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
index c88fddb..e1bcd8a 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
 
 import java.io.Serializable;
 import java.util.Date;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
similarity index 95%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
index 91592e7..fda814c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
 
 public class EventList {
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
similarity index 96%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
index 56dfc8a..4fde136 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
 
 import java.io.Serializable;
 
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java
new file mode 100644
index 0000000..6744746
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java
@@ -0,0 +1,78 @@
+package edu.uci.ics.asterix.event.service;
+
+import java.io.File;
+import java.io.FileFilter;
+
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+import edu.uci.ics.asterix.event.error.OutputHandler;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
+
+public class AsterixEventService {
+
+    private static final Logger LOGGER = Logger.getLogger(AsterixEventService.class.getName());
+    private static Configuration configuration;
+    private static String asterixDir;
+    private static String asterixZip;
+    private static String eventHome;
+
+    public static void initialize(Configuration configuration, String asterixDir, String eventHome) throws Exception {
+        AsterixEventService.configuration = configuration;
+        AsterixEventService.asterixDir = asterixDir;
+        AsterixEventService.asterixZip = initBinary("asterix-server");
+        AsterixEventService.eventHome = eventHome;
+    }
+
+    public static AsterixEventServiceClient getAsterixEventServiceClient(Cluster cluster, boolean transferArtifacts,
+            boolean dryRun) throws Exception {
+        AsterixEventServiceClient client = new AsterixEventServiceClient(configuration, eventHome, cluster,
+                transferArtifacts, dryRun, OutputHandler.INSTANCE);
+        return client;
+    }
+
+    public static AsterixEventServiceClient getAsterixEventServiceClient(Cluster cluster) throws Exception {
+        AsterixEventServiceClient client = new AsterixEventServiceClient(configuration, eventHome, cluster, false,
+                false, OutputHandler.INSTANCE);
+        return client;
+    }
+
+    private static String initBinary(final String fileNamePattern) {
+        File file = new File(asterixDir);
+        File[] zipFiles = file.listFiles(new FileFilter() {
+            public boolean accept(File arg0) {
+                return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
+            }
+        });
+        if (zipFiles.length == 0) {
+            String msg = " Binary not found at " + asterixDir;
+            LOGGER.log(Level.FATAL, msg);
+            throw new IllegalStateException(msg);
+        }
+        if (zipFiles.length > 1) {
+            String msg = " Multiple binaries found at " + asterixDir;
+            LOGGER.log(Level.FATAL, msg);
+            throw new IllegalStateException(msg);
+        }
+
+        return zipFiles[0].getAbsolutePath();
+    }
+
+    public static Configuration getConfiguration() {
+        return configuration;
+    }
+
+    public static String getAsterixZip() {
+        return asterixZip;
+    }
+
+    public static String getAsterixDir() {
+        return asterixDir;
+    }
+
+    public static String getEventHome() {
+        return eventHome;
+    }
+}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java
new file mode 100644
index 0000000..2a70862
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java
@@ -0,0 +1,551 @@
+/*
+ * Copyright 2009-2012 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.event.service;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.jar.JarOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipOutputStream;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+
+import org.apache.commons.io.IOUtils;
+
+import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.common.configuration.Coredump;
+import edu.uci.ics.asterix.common.configuration.Store;
+import edu.uci.ics.asterix.common.configuration.TransactionLogDir;
+import edu.uci.ics.asterix.event.driver.EventDriver;
+import edu.uci.ics.asterix.event.error.EventException;
+import edu.uci.ics.asterix.event.management.EventUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Env;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.cluster.Property;
+
+public class AsterixEventServiceUtil {
+
+    public static final String TXN_LOG_DIR = "txnLogs";
+    public static final String TXN_LOG_DIR_KEY_SUFFIX = "txnLogDir";
+    public static final String ASTERIX_CONFIGURATION_FILE = "asterix-configuration.xml";
+    public static final String TXN_LOG_CONFIGURATION_FILE = "log.properties";
+    public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
+    public static final String ASTERIX_DIR = "asterix";
+    public static final String EVENT_DIR = "events";
+    public static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator + File.separator
+            + "asterix-configuration.xml";
+    public static final int CLUSTER_NET_PORT_DEFAULT = 1098;
+    public static final int CLIENT_NET_PORT_DEFAULT = 1099;
+    public static final int HTTP_PORT_DEFAULT = 8888;
+    public static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
+
+    public static final String MANAGIX_INTERNAL_DIR = ".installer";
+    public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+
+    public static AsterixInstance createAsterixInstance(String asterixInstanceName, Cluster cluster,
+            AsterixConfiguration asterixConfiguration) throws FileNotFoundException, IOException {
+        Node metadataNode = getMetadataNode(asterixInstanceName, cluster);
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        String asterixVersion = asterixZipName.substring("asterix-server-".length(),
+                asterixZipName.indexOf("-binary-assembly"));
+        AsterixInstance instance = new AsterixInstance(asterixInstanceName, cluster, asterixConfiguration,
+                metadataNode.getId(), asterixVersion);
+        return instance;
+    }
+
+    public static void createAsterixZip(AsterixInstance asterixInstance) throws IOException, InterruptedException,
+            JAXBException, EventException {
+
+        String modifiedZipPath = injectAsterixPropertyFile(AsterixEventService.getAsterixZip(), asterixInstance);
+        injectAsterixClusterConfigurationFile(modifiedZipPath, asterixInstance);
+    }
+
+    public static void createClusterProperties(Cluster cluster, AsterixConfiguration asterixConfiguration) {
+
+        String ccJavaOpts = null;
+        String ncJavaOpts = null;
+        for (edu.uci.ics.asterix.common.configuration.Property property : asterixConfiguration.getProperty()) {
+            if (property.getName().equalsIgnoreCase(EventUtil.CC_JAVA_OPTS)) {
+                ccJavaOpts = property.getValue();
+            } else if (property.getName().equalsIgnoreCase(EventUtil.NC_JAVA_OPTS)) {
+                ncJavaOpts = property.getValue();
+            }
+        }
+
+        poulateClusterEnvironmentProperties(cluster, ccJavaOpts, ncJavaOpts);
+    }
+
+    public static void poulateClusterEnvironmentProperties(Cluster cluster, String ccJavaOpts, String ncJavaOpts) {
+        List<Property> clusterProperties = null;
+        if (cluster.getEnv() != null && cluster.getEnv().getProperty() != null) {
+            clusterProperties = cluster.getEnv().getProperty();
+            clusterProperties.clear();
+        } else {
+            clusterProperties = new ArrayList<Property>();
+        }
+
+        clusterProperties.add(new Property(EventUtil.CC_JAVA_OPTS, ccJavaOpts));
+        clusterProperties.add(new Property(EventUtil.NC_JAVA_OPTS, ncJavaOpts));
+        clusterProperties.add(new Property("ASTERIX_HOME", cluster.getWorkingDir().getDir() + File.separator
+                + "asterix"));
+        clusterProperties.add(new Property("LOG_DIR", cluster.getLogDir()));
+        clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
+        clusterProperties.add(new Property("WORKING_DIR", cluster.getWorkingDir().getDir()));
+        clusterProperties.add(new Property("CLIENT_NET_IP", cluster.getMasterNode().getClientIp()));
+        clusterProperties.add(new Property("CLUSTER_NET_IP", cluster.getMasterNode().getClusterIp()));
+
+        int clusterNetPort = cluster.getMasterNode().getClusterPort() != null ? cluster.getMasterNode()
+                .getClusterPort().intValue() : CLUSTER_NET_PORT_DEFAULT;
+        int clientNetPort = cluster.getMasterNode().getClientPort() != null ? cluster.getMasterNode().getClientPort()
+                .intValue() : CLIENT_NET_PORT_DEFAULT;
+        int httpPort = cluster.getMasterNode().getHttpPort() != null ? cluster.getMasterNode().getHttpPort().intValue()
+                : HTTP_PORT_DEFAULT;
+
+        clusterProperties.add(new Property("CLIENT_NET_PORT", "" + clientNetPort));
+        clusterProperties.add(new Property("CLUSTER_NET_PORT", "" + clusterNetPort));
+        clusterProperties.add(new Property("HTTP_PORT", "" + httpPort));
+
+        cluster.setEnv(new Env(clusterProperties));
+    }
+
+    private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+            throws IOException, JAXBException {
+        writeAsterixConfigurationFile(asterixInstance);
+        String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+        unzip(origZipFile, asterixInstanceDir);
+        File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        File replacementFile = new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE);
+        replaceInJar(sourceJar, ASTERIX_CONFIGURATION_FILE, replacementFile);
+        new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE).delete();
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
+    }
+
+    private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+            throws IOException, EventException {
+        String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+        unzip(origZipFile, asterixInstanceDir);
+        File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        Properties txnLogProperties = new Properties();
+        URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
+        InputStream in = urlClassLoader.getResourceAsStream(TXN_LOG_CONFIGURATION_FILE);
+        if (in != null) {
+            txnLogProperties.load(in);
+        }
+
+        writeAsterixLogConfigurationFile(asterixInstance, txnLogProperties);
+
+        File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
+        replaceInJar(sourceJar2, TXN_LOG_CONFIGURATION_FILE, replacementFile);
+
+        new File(asterixInstanceDir + File.separator + "log.properties").delete();
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
+    }
+
+    private static String injectAsterixClusterConfigurationFile(String origZipFile, AsterixInstance asterixInstance)
+            throws IOException, EventException, JAXBException {
+        String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+        unzip(origZipFile, asterixInstanceDir);
+        File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        writeAsterixClusterConfigurationFile(asterixInstance);
+
+        File replacementFile = new File(asterixInstanceDir + File.separator + "cluster.xml");
+        replaceInJar(sourceJar, CLUSTER_CONFIGURATION_FILE, replacementFile);
+
+        new File(asterixInstanceDir + File.separator + CLUSTER_CONFIGURATION_FILE).delete();
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
+    }
+
+    private static void writeAsterixClusterConfigurationFile(AsterixInstance asterixInstance) throws IOException,
+            EventException, JAXBException {
+        String asterixInstanceName = asterixInstance.getName();
+        Cluster cluster = asterixInstance.getCluster();
+
+        JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+        Marshaller marshaller = ctx.createMarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        marshaller.marshal(cluster, new FileOutputStream(AsterixEventService.getAsterixDir() + File.separator
+                + asterixInstanceName + File.separator + "cluster.xml"));
+    }
+
+    public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
+            String libraryName, String libraryPath) throws IOException {
+        File instanceDir = new File(AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName());
+        if (!instanceDir.exists()) {
+            instanceDir.mkdirs();
+        }
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+
+        String sourceZip = instanceDir.getAbsolutePath() + File.separator + asterixZipName;
+        unzip(sourceZip, instanceDir.getAbsolutePath());
+        File libraryPathInZip = new File(instanceDir.getAbsolutePath() + File.separator + "external" + File.separator
+                + "library" + dataverseName + File.separator + "to-add" + File.separator + libraryName);
+        libraryPathInZip.mkdirs();
+        Runtime.getRuntime().exec("cp" + " " + libraryPath + " " + libraryPathInZip.getAbsolutePath());
+        Runtime.getRuntime().exec("rm " + sourceZip);
+        String destZip = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName()
+                + File.separator + asterixZipName;
+        zipDir(instanceDir, new File(destZip));
+        Runtime.getRuntime().exec("mv" + " " + destZip + " " + sourceZip);
+    }
+
+    private static Node getMetadataNode(String asterixInstanceName, Cluster cluster) {
+        Node metadataNode = null;
+        if (cluster.getMetadataNode() != null) {
+            for(Node node: cluster.getNode()){
+                if(node.getId().equals(cluster.getMetadataNode())){
+                    metadataNode = node;
+                    break;
+                }
+            }
+        } else {
+            Random random = new Random();
+            int nNodes = cluster.getNode().size();
+            metadataNode = cluster.getNode().get(random.nextInt(nNodes));
+        }
+        return metadataNode;
+    }
+
+    public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
+        String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
+        String[] storeDirs = null;
+        StringBuffer nodeDataStore = new StringBuffer();
+        String storeDirValue = node.getStore();
+        if (storeDirValue == null) {
+            storeDirValue = cluster.getStore();
+            if (storeDirValue == null) {
+                throw new IllegalStateException(" Store not defined for node " + node.getId());
+            }
+            storeDataSubDir = node.getId() + File.separator + storeDataSubDir;
+        }
+
+        storeDirs = storeDirValue.split(",");
+        for (String ns : storeDirs) {
+            nodeDataStore.append(ns + File.separator + storeDataSubDir.trim());
+            nodeDataStore.append(",");
+        }
+        nodeDataStore.deleteCharAt(nodeDataStore.length() - 1);
+        return nodeDataStore.toString();
+    }
+
+    private static void writeAsterixConfigurationFile(AsterixInstance asterixInstance) throws IOException,
+            JAXBException {
+        String asterixInstanceName = asterixInstance.getName();
+        Cluster cluster = asterixInstance.getCluster();
+        String metadataNodeId = asterixInstance.getMetadataNodeId();
+
+        AsterixConfiguration configuration = asterixInstance.getAsterixConfiguration();
+        configuration.setInstanceName(asterixInstanceName);
+        configuration.setMetadataNode(asterixInstanceName + "_" + metadataNodeId);
+        String storeDir = null;
+        List<Store> stores = new ArrayList<Store>();
+        for (Node node : cluster.getNode()) {
+            storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+            stores.add(new Store(asterixInstanceName + "_" + node.getId(), storeDir));
+        }
+        configuration.setStore(stores);
+
+        List<Coredump> coredump = new ArrayList<Coredump>();
+        String coredumpDir = null;
+        List<TransactionLogDir> txnLogDirs = new ArrayList<TransactionLogDir>();
+        String txnLogDir = null;
+        for (Node node : cluster.getNode()) {
+            coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+            coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator
+                    + asterixInstanceName + "_" + node.getId()));
+
+            txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+            txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
+        }
+        configuration.setCoredump(coredump);
+        configuration.setTransactionLogDir(txnLogDirs);
+
+        File asterixConfDir = new File(AsterixEventService.getAsterixDir() + File.separator + asterixInstanceName);
+        asterixConfDir.mkdirs();
+
+        JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
+        Marshaller marshaller = ctx.createMarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        FileOutputStream os = new FileOutputStream(asterixConfDir + File.separator + ASTERIX_CONFIGURATION_FILE);
+        marshaller.marshal(configuration, os);
+        os.close();
+    }
+
+    private static void writeAsterixLogConfigurationFile(AsterixInstance asterixInstance, Properties logProperties)
+            throws IOException, EventException {
+        String asterixInstanceName = asterixInstance.getName();
+        Cluster cluster = asterixInstance.getCluster();
+        StringBuffer conf = new StringBuffer();
+        for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
+            conf.append(p.getKey() + "=" + p.getValue() + "\n");
+        }
+
+        for (Node node : cluster.getNode()) {
+            String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+            if (txnLogDir == null) {
+                throw new EventException("Transaction log directory (txn_log_dir) not configured for node: "
+                        + node.getId());
+            }
+            conf.append(asterixInstanceName + "_" + node.getId() + "." + TXN_LOG_DIR_KEY_SUFFIX + "=" + txnLogDir
+                    + "\n");
+        }
+        List<edu.uci.ics.asterix.common.configuration.Property> properties = asterixInstance.getAsterixConfiguration()
+                .getProperty();
+        for (edu.uci.ics.asterix.common.configuration.Property p : properties) {
+            if (p.getName().trim().toLowerCase().contains("log")) {
+                conf.append(p.getValue() + "=" + p.getValue());
+            }
+        }
+        dumpToFile(AsterixEventService.getAsterixDir() + File.separator + asterixInstanceName + File.separator
+                + "log.properties", conf.toString());
+
+    }
+
+    public static void unzip(String sourceFile, String destDir) throws IOException {
+        BufferedOutputStream dest = null;
+        FileInputStream fis = new FileInputStream(sourceFile);
+        ZipInputStream zis = new ZipInputStream(new BufferedInputStream(fis));
+        ZipEntry entry = null;
+
+        int BUFFER_SIZE = 4096;
+        while ((entry = zis.getNextEntry()) != null) {
+            String dst = destDir + File.separator + entry.getName();
+            if (entry.isDirectory()) {
+                createDir(destDir, entry);
+                continue;
+            }
+            int count;
+            byte data[] = new byte[BUFFER_SIZE];
+
+            // write the file to the disk
+            FileOutputStream fos = new FileOutputStream(dst);
+            dest = new BufferedOutputStream(fos, BUFFER_SIZE);
+            while ((count = zis.read(data, 0, BUFFER_SIZE)) != -1) {
+                dest.write(data, 0, count);
+            }
+            // close the output streams
+            dest.flush();
+            dest.close();
+        }
+
+        zis.close();
+    }
+
+    public static void zipDir(File sourceDir, File destFile) throws IOException {
+        FileOutputStream fos = new FileOutputStream(destFile);
+        ZipOutputStream zos = new ZipOutputStream(fos);
+        zipDir(sourceDir, destFile, zos);
+        zos.close();
+    }
+
+    private static void zipDir(File sourceDir, final File destFile, ZipOutputStream zos) throws IOException {
+        File[] dirList = sourceDir.listFiles(new FileFilter() {
+            public boolean accept(File f) {
+                return !f.getName().endsWith(destFile.getName());
+            }
+        });
+        for (int i = 0; i < dirList.length; i++) {
+            File f = dirList[i];
+            if (f.isDirectory()) {
+                zipDir(f, destFile, zos);
+            } else {
+                int bytesIn = 0;
+                byte[] readBuffer = new byte[2156];
+                FileInputStream fis = new FileInputStream(f);
+                ZipEntry entry = new ZipEntry(sourceDir.getName() + File.separator + f.getName());
+                zos.putNextEntry(entry);
+                while ((bytesIn = fis.read(readBuffer)) != -1) {
+                    zos.write(readBuffer, 0, bytesIn);
+                }
+                fis.close();
+            }
+        }
+    }
+
+    private static void replaceInJar(File sourceJar, String origFile, File replacementFile) throws IOException {
+        String srcJarAbsPath = sourceJar.getAbsolutePath();
+        String srcJarSuffix = srcJarAbsPath.substring(srcJarAbsPath.lastIndexOf(File.separator) + 1);
+        String srcJarName = srcJarSuffix.split(".jar")[0];
+
+        String destJarName = srcJarName + "-managix";
+        String destJarSuffix = destJarName + ".jar";
+        File destJar = new File(sourceJar.getParentFile().getAbsolutePath() + File.separator + destJarSuffix);
+        //  File destJar = new File(sourceJar.getAbsolutePath() + ".modified");
+        InputStream jarIs = null;
+        FileInputStream fis = new FileInputStream(replacementFile);
+        JarFile sourceJarFile = new JarFile(sourceJar);
+        Enumeration<JarEntry> entries = sourceJarFile.entries();
+        JarOutputStream jos = new JarOutputStream(new FileOutputStream(destJar));
+        byte[] buffer = new byte[2048];
+        int read;
+        while (entries.hasMoreElements()) {
+            JarEntry entry = (JarEntry) entries.nextElement();
+            String name = entry.getName();
+            if (name.equals(origFile)) {
+                continue;
+            }
+            jarIs = sourceJarFile.getInputStream(entry);
+            jos.putNextEntry(entry);
+            while ((read = jarIs.read(buffer)) != -1) {
+                jos.write(buffer, 0, read);
+            }
+        }
+        JarEntry entry = new JarEntry(origFile);
+        jos.putNextEntry(entry);
+        while ((read = fis.read(buffer)) != -1) {
+            jos.write(buffer, 0, read);
+        }
+        fis.close();
+        jos.close();
+        jarIs.close();
+        sourceJar.delete();
+        destJar.renameTo(sourceJar);
+        destJar.setExecutable(true);
+    }
+
+    public static void dumpToFile(String dest, String content) throws IOException {
+        FileWriter writer = new FileWriter(dest);
+        writer.write(content);
+        writer.close();
+    }
+
+    private static void createDir(String destDirectory, ZipEntry entry) {
+        String name = entry.getName();
+        int index = name.lastIndexOf(File.separator);
+        String dirSequence = name.substring(0, index);
+        File newDirs = new File(destDirectory + File.separator + dirSequence);
+        newDirs.mkdirs();
+    }
+
+    public static AsterixInstance validateAsterixInstanceExists(String name, State... permissibleStates)
+            throws Exception {
+        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
+        if (instance == null) {
+            throw new EventException("Asterix instance by name " + name + " does not exist.");
+        }
+        boolean valid = false;
+        for (State state : permissibleStates) {
+            if (state.equals(instance.getState())) {
+                valid = true;
+                break;
+            }
+        }
+        if (!valid) {
+            throw new EventException("Asterix instance by the name " + name + " is in " + instance.getState()
+                    + " state ");
+        }
+        return instance;
+    }
+
+    public static void validateAsterixInstanceNotExists(String name) throws Exception {
+        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
+        if (instance != null) {
+            throw new EventException("Asterix instance by name " + name + " already exists.");
+        }
+    }
+
+    public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
+        List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
+        List<String> usedIps = new ArrayList<String>();
+        String masterIp = instance.getCluster().getMasterNode().getClusterIp();
+        for (Node node : instance.getCluster().getNode()) {
+            usedIps.add(node.getClusterIp());
+        }
+        usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
+        boolean conflictFound = false;
+        AsterixInstance conflictingInstance = null;
+        for (AsterixInstance existing : existingInstances) {
+            conflictFound = !existing.getState().equals(State.INACTIVE)
+                    && existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
+            if (conflictFound) {
+                conflictingInstance = existing;
+                break;
+            }
+            for (Node n : existing.getCluster().getNode()) {
+                if (usedIps.contains(n.getClusterIp())) {
+                    conflictFound = true;
+                    conflictingInstance = existing;
+                    break;
+                }
+            }
+        }
+        if (conflictFound) {
+            throw new Exception("Cluster definition conflicts with an existing instance of Asterix: "
+                    + conflictingInstance.getName());
+        }
+    }
+
+    public static void deleteDirectory(String path) throws IOException {
+        Runtime.getRuntime().exec("rm -rf " + path);
+    }
+
+    public static String executeLocalScript(String path, List<String> args) throws Exception {
+        List<String> pargs = new ArrayList<String>();
+        pargs.add("/bin/bash");
+        pargs.add(path);
+        if (args != null) {
+            pargs.addAll(args);
+        }
+        ProcessBuilder pb = new ProcessBuilder(pargs);
+        pb.environment().putAll(EventDriver.getEnvironment());
+        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
+        Process p = pb.start();
+        BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
+        StringWriter writer = new StringWriter();
+        IOUtils.copy(bis, writer, "UTF-8");
+        return writer.toString();
+    }
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
similarity index 92%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
index 59fa198..cac504f 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
@@ -12,11 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
 
 import java.util.List;
 
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 
 public interface ILookupService {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
similarity index 94%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
index 98d72f4..56e5337 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
 
 public class ServiceProvider {
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
similarity index 91%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
index d3304d2..5c059bb 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -34,10 +34,8 @@
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
+import edu.uci.ics.asterix.event.error.EventException;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 
 public class ZooKeeperService implements ILookupService {
@@ -47,8 +45,7 @@
     private static final int ZOOKEEPER_LEADER_CONN_PORT = 2222;
     private static final int ZOOKEEPER_LEADER_ELEC_PORT = 2223;
     private static final int ZOOKEEPER_SESSION_TIME_OUT = 40 * 1000; //milliseconds
-    private static final String ZOOKEEPER_HOME = InstallerDriver.getManagixHome() + File.separator
-            + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "zookeeper";
+    private static final String ZOOKEEPER_HOME = AsterixEventService.getEventHome() + File.separator + "zookeeper";
     private static final String ZOO_KEEPER_CONFIG = ZOOKEEPER_HOME + File.separator + "zk.cfg";
 
     private boolean isRunning = false;
@@ -109,10 +106,10 @@
         if (head == null) {
             StringBuilder msg = new StringBuilder(
                     "Unable to start Zookeeper Service. This could be because of the following reasons.\n");
-            msg.append("1) Managix is incorrectly configured. Please run " + InstallerDriver.getManagixHome()
-                    + "/bin/managix validate" + " to run a validation test and correct the errors reported.");
+            msg.append("1) Managix is incorrectly configured. Please run " + "managix validate"
+                    + " to run a validation test and correct the errors reported.");
             msg.append("\n2) If validation in (1) is successful, ensure that java_home parameter is set correctly in Managix configuration ("
-                    + InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_CONF_XML + ")");
+                    + AsterixEventServiceUtil.MANAGIX_CONF_XML + ")");
             throw new Exception(msg.toString());
         }
         msgQ.take();
@@ -172,7 +169,7 @@
 
     public void removeAsterixInstance(String name) throws Exception {
         if (!exists(name)) {
-            throw new InstallerException("Asterix instance by name " + name + " does not exists.");
+            throw new EventException("Asterix instance by name " + name + " does not exists.");
         }
         zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name, DEFAULT_NODE_VERSION);
     }
@@ -244,7 +241,7 @@
             buffer.append("server" + "." + serverId + "=" + server + ":" + leaderConnPort + ":" + leaderElecPort + "\n");
             serverId++;
         }
-        InstallerUtil.dumpToFile(zooKeeperConfigPath, buffer.toString());
+        AsterixEventServiceUtil.dumpToFile(zooKeeperConfigPath, buffer.toString());
     }
 
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java
new file mode 100644
index 0000000..ff600ce
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.event.util;
+
+public class AsterixConstants {
+
+    public static String ASTERIX_ROOT_METADATA_DIR = "asterix_root_metadata";
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
similarity index 72%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
index fe61462..e06d66c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
@@ -12,17 +12,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.installer.events;
+package edu.uci.ics.asterix.event.util;
 
 import java.io.File;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import edu.uci.ics.asterix.event.driver.EventDriver;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.BackupInfo;
+import edu.uci.ics.asterix.event.model.BackupInfo.BackupType;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
 import edu.uci.ics.asterix.event.schema.pattern.Delay;
@@ -31,20 +36,20 @@
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.event.schema.pattern.Value;
-import edu.uci.ics.asterix.installer.command.BackupCommand;
-import edu.uci.ics.asterix.installer.command.StopCommand;
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
-import edu.uci.ics.asterix.installer.model.BackupInfo.BackupType;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.schema.conf.Backup;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class PatternCreator {
 
+    public static PatternCreator INSTANCE = new PatternCreator();
+
+    private PatternCreator() {
+
+    }
+
     private ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
 
     private void addInitialDelay(Pattern p, int delay, String unit) {
@@ -94,9 +99,9 @@
         return patterns;
     }
 
-    public Patterns getStopCommandPattern(StopCommand stopCommand) throws Exception {
+    public Patterns getStopCommandPattern(String asterixInstanceName) throws Exception {
         List<Pattern> ps = new ArrayList<Pattern>();
-        AsterixInstance asterixInstance = lookupService.getAsterixInstance(stopCommand.getAsterixInstanceName());
+        AsterixInstance asterixInstance = lookupService.getAsterixInstance(asterixInstanceName);
         Cluster cluster = asterixInstance.getCluster();
 
         String ccLocation = cluster.getMasterNode().getId();
@@ -104,7 +109,6 @@
         addInitialDelay(createCC, 5, "sec");
         ps.add(createCC);
 
-        String asterixInstanceName = stopCommand.getAsterixInstanceName();
         int nodeControllerIndex = 1;
         for (Node node : cluster.getNode()) {
             Pattern createNC = createNCStopPattern(node.getId(), asterixInstanceName + "_" + nodeControllerIndex);
@@ -161,8 +165,9 @@
             iodevices = node.getIodevices() == null ? instance.getCluster().getIodevices() : node.getIodevices();
             store = node.getStore() == null ? cluster.getStore() : node.getStore();
             pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
-                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
-                    + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+                    + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+                    + backupId + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " "
+                    + hadoopVersion;
             Event event = new Event("backup", nodeid, pargs);
             patternList.add(new Pattern(null, 1, null, event));
         }
@@ -185,7 +190,7 @@
             txnLogDir = node.getTxnLogDir() == null ? instance.getCluster().getTxnLogDir() : node.getTxnLogDir();
             store = node.getStore() == null ? cluster.getStore() : node.getStore();
             pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
-                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR  + " " + txnLogDir + " " + backupId + " " + backupDir
+                    + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + txnLogDir + " " + backupId + " " + backupDir
                     + " " + "local" + " " + node.getId();
             Event event = new Event("backup", nodeid, pargs);
             patternList.add(new Pattern(null, 1, null, event));
@@ -210,8 +215,9 @@
             String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
             nodeStore = node.getStore() == null ? clusterStore : node.getStore();
             pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
-                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
-                    + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+                    + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+                    + backupId + " " + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " "
+                    + hadoopVersion;
             Event event = new Event("restore", nodeid, pargs);
             patternList.add(new Pattern(null, 1, null, event));
         }
@@ -232,8 +238,8 @@
             String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
             nodeStore = node.getStore() == null ? clusterStore : node.getStore();
             pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
-                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
-                    + backupDir + " " + "local" + " " + node.getId();
+                    + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+                    + backupId + " " + backupDir + " " + "local" + " " + node.getId();
             Event event = new Event("restore", nodeid, pargs);
             patternList.add(new Pattern(null, 1, null, event));
         }
@@ -243,9 +249,8 @@
     public Patterns createHadoopLibraryTransferPattern(Cluster cluster) throws Exception {
         List<Pattern> patternList = new ArrayList<Pattern>();
         String workingDir = cluster.getWorkingDir().getDir();
-        String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
-        File hadoopDir = new File(InstallerDriver.getManagixHome() + File.separator
-                + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "hadoop-" + hadoopVersion);
+        String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
+        File hadoopDir = new File(AsterixEventService.getEventHome() + File.separator + "hadoop-" + hadoopVersion);
         if (!hadoopDir.exists()) {
             throw new IllegalStateException("Hadoop version :" + hadoopVersion + " not supported");
         }
@@ -332,8 +337,8 @@
     private Patterns createRemoveHDFSBackupPattern(AsterixInstance instance, String hdfsBackupDir) throws Exception {
         List<Pattern> patternList = new ArrayList<Pattern>();
         Cluster cluster = instance.getCluster();
-        String hdfsUrl = InstallerDriver.getConfiguration().getBackup().getHdfs().getUrl();
-        String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
+        String hdfsUrl = AsterixEventService.getConfiguration().getBackup().getHdfs().getUrl();
+        String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
         String workingDir = cluster.getWorkingDir().getDir();
         Node launchingNode = cluster.getNode().get(0);
         Nodeid nodeid = new Nodeid(new Value(null, launchingNode.getId()));
@@ -386,6 +391,85 @@
         return patterns;
     }
 
+    public Patterns getLibraryInstallPattern(AsterixInstance instance, String dataverse, String libraryName,
+            String libraryPath) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+        String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+        String workingDir = cluster.getWorkingDir().getDir();
+        String destDir = workingDir + File.separator + "library" + File.separator + dataverse + File.separator
+                + libraryName;
+        String fileToTransfer = new File(libraryPath).getAbsolutePath();
+
+        Iterator<Node> installTargets = cluster.getNode().iterator();
+        Node installNode = installTargets.next();
+        String destinationIp = installNode.getClusterIp();
+        String pargs = username + " " + fileToTransfer + " " + destinationIp + " " + destDir + " " + "unpack";
+        Event event = new Event("file_transfer", nodeid, pargs);
+        Pattern p = new Pattern(null, 1, null, event);
+        patternList.add(p);
+
+        if (!cluster.getWorkingDir().isNFS()) {
+            while (installTargets.hasNext()) {
+                Node node = installTargets.next();
+                pargs = username + " " + fileToTransfer + " " + node.getClusterIp() + " " + destDir + " " + "unpack";
+                event = new Event("file_transfer", nodeid, pargs);
+                p = new Pattern(null, 1, null, event);
+                patternList.add(p);
+            }
+
+            pargs = username + " " + fileToTransfer + " " + cluster.getMasterNode().getClusterIp() + " " + destDir
+                    + " " + "unpack";
+            event = new Event("file_transfer", nodeid, pargs);
+            p = new Pattern(null, 1, null, event);
+            patternList.add(p);
+        }
+        return new Patterns(patternList);
+    }
+
+    public Patterns getLibraryUninstallPattern(AsterixInstance instance, String dataverse, String libraryName)
+            throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        String workingDir = cluster.getWorkingDir().getDir();
+        String destFile = dataverse + "." + libraryName;
+        String pargs = workingDir + File.separator + "uninstall" + " " + destFile;
+
+        String metadataNodeId = instance.getMetadataNodeId();
+        Nodeid nodeid = new Nodeid(new Value(null, metadataNodeId));
+        Event event = new Event("file_create", nodeid, pargs);
+        Pattern p = new Pattern(null, 1, null, event);
+        patternList.add(p);
+
+        Iterator<Node> uninstallTargets = cluster.getNode().iterator();
+        String libDir = workingDir + File.separator + "library" + File.separator + dataverse + File.separator
+                + libraryName;
+        Node uninstallNode = uninstallTargets.next();
+        nodeid = new Nodeid(new Value(null, uninstallNode.getId()));
+        event = new Event("file_delete", nodeid, libDir);
+        p = new Pattern(null, 1, null, event);
+        patternList.add(p);
+        pargs = libDir;
+
+        if (!cluster.getWorkingDir().isNFS()) {
+            while (uninstallTargets.hasNext()) {
+                uninstallNode = uninstallTargets.next();
+                nodeid = new Nodeid(new Value(null, uninstallNode.getId()));
+                event = new Event("file_delete", nodeid, pargs);
+                p = new Pattern(null, 1, null, event);
+                patternList.add(p);
+            }
+
+            nodeid = new Nodeid(new Value(null, cluster.getMasterNode().getId()));
+            event = new Event("file_delete", nodeid, pargs);
+            p = new Pattern(null, 1, null, event);
+            patternList.add(p);
+
+        }
+        return new Patterns(patternList);
+    }
+
     private Patterns createRemoveAsterixRootMetadata(AsterixInstance instance) throws Exception {
         List<Pattern> patternList = new ArrayList<Pattern>();
         Cluster cluster = instance.getCluster();
@@ -395,7 +479,7 @@
         for (Node node : cluster.getNode()) {
             String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
             String primaryIODevice = iodevices.split(",")[0].trim();
-            pargs = primaryIODevice + File.separator + BackupCommand.ASTERIX_ROOT_METADATA_DIR;
+            pargs = primaryIODevice + File.separator + AsterixConstants.ASTERIX_ROOT_METADATA_DIR;
             nodeid = new Nodeid(new Value(null, node.getId()));
             event = new Event("file_delete", nodeid, pargs);
             patternList.add(new Pattern(null, 1, null, event));
@@ -447,9 +531,9 @@
     private Pattern createCopyHyracksPattern(String instanceName, Cluster cluster, String destinationIp, String destDir) {
         Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
         String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
-        String asterixZipName = InstallerDriver.getAsterixZip().substring(
-                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-        String fileToTransfer = new File(InstallerDriver.getAsterixDir() + File.separator + instanceName
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        String fileToTransfer = new File(AsterixEventService.getAsterixDir() + File.separator + instanceName
                 + File.separator + asterixZipName).getAbsolutePath();
         String pargs = username + " " + fileToTransfer + " " + destinationIp + " " + destDir + " " + "unpack";
         Event event = new Event("file_transfer", nodeid, pargs);
@@ -481,10 +565,48 @@
         return new Pattern(null, 1, null, event);
     }
 
+    public Patterns createPrepareNodePattern(String instanceName, Cluster cluster, Node nodeToBeAdded) {
+        List<Pattern> ps = new ArrayList<Pattern>();
+        boolean workingDirOnNFS = cluster.getWorkingDir().isNFS();
+        if (!workingDirOnNFS) {
+            String ccLocationIp = cluster.getMasterNode().getClusterIp();
+            String destDir = cluster.getWorkingDir().getDir() + File.separator + "asterix";
+            Pattern copyHyracks = createCopyHyracksPattern(instanceName, cluster, ccLocationIp, destDir);
+            ps.add(copyHyracks);
+
+            String workingDir = cluster.getWorkingDir().getDir();
+            String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
+            File hadoopDir = new File(AsterixEventService.getEventHome() + File.separator + "hadoop-" + hadoopVersion);
+            if (!hadoopDir.exists()) {
+                throw new IllegalStateException("Hadoop version :" + hadoopVersion + " not supported");
+            }
+
+            Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+            String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+            String pargs = username + " " + hadoopDir.getAbsolutePath() + " " + cluster.getMasterNode().getClusterIp()
+                    + " " + workingDir;
+            Event event = new Event("directory_transfer", nodeid, pargs);
+            Pattern p = new Pattern(null, 1, null, event);
+            addInitialDelay(p, 2, "sec");
+            ps.add(p);
+
+            nodeid = new Nodeid(new Value(null, nodeToBeAdded.getId()));
+            pargs = cluster.getUsername() + " " + hadoopDir.getAbsolutePath() + " " + nodeToBeAdded.getClusterIp()
+                    + " " + workingDir;
+            event = new Event("directory_transfer", nodeid, pargs);
+            p = new Pattern(null, 1, null, event);
+            addInitialDelay(p, 2, "sec");
+            ps.add(p);
+        }
+
+        Patterns patterns = new Patterns(ps);
+        return patterns;
+    }
+
     public Patterns getGenerateLogPattern(String asterixInstanceName, Cluster cluster, String outputDir) {
         List<Pattern> patternList = new ArrayList<Pattern>();
-        Map<String,String> nodeLogs = new HashMap<String,String>();
-        
+        Map<String, String> nodeLogs = new HashMap<String, String>();
+
         String username = cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername();
         String srcHost = cluster.getMasterNode().getClientIp();
         Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
@@ -495,15 +617,15 @@
         Event event = new Event("directory_copy", nodeid, pargs);
         Pattern p = new Pattern(null, 1, null, event);
         patternList.add(p);
-        nodeLogs.put(cluster.getMasterNode().getClusterIp(),srcDir);
+        nodeLogs.put(cluster.getMasterNode().getClusterIp(), srcDir);
         for (Node node : cluster.getNode()) {
             srcHost = node.getClusterIp();
             srcDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
-            if(nodeLogs.get(node.getClusterIp()) != null && nodeLogs.get(node.getClusterIp()).equals(srcDir)){
+            if (nodeLogs.get(node.getClusterIp()) != null && nodeLogs.get(node.getClusterIp()).equals(srcDir)) {
                 continue;
             }
             destDir = outputDir + File.separator + node.getId();
-            pargs = username + " " + srcHost +  " " + srcDir + " "  + destDir;
+            pargs = username + " " + srcHost + " " + srcDir + " " + destDir;
             event = new Event("directory_copy", nodeid, pargs);
             p = new Pattern(null, 1, null, event);
             patternList.add(p);
diff --git a/asterix-events/src/main/resources/events/backup/backup.sh b/asterix-events/src/main/resources/events/backup/backup.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/cc_failure/cc_failure.sh b/asterix-events/src/main/resources/events/cc_failure/cc_failure.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/cc_start/cc_start.sh b/asterix-events/src/main/resources/events/cc_start/cc_start.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/events.xml b/asterix-events/src/main/resources/events/events.xml
index 01495cb..67a29a2 100644
--- a/asterix-events/src/main/resources/events/events.xml
+++ b/asterix-events/src/main/resources/events/events.xml
@@ -94,7 +94,14 @@
     <type>file_delete</type>
     <script>file/delete.sh</script>
     <description>Deletes a file on the local file system to a remote node</description>
-    <args>local_source_path destination_node destination_path</args>
+    <args>destination_node destination_path</args>
+    <daemon>false</daemon>
+  </event>
+  <event>
+    <type>file_create</type>
+    <script>file/create_file.sh</script>
+    <description>Creates a file on the local file system to a remote node</description>
+    <args>destination_node destination_path</args>
     <daemon>false</daemon>
   </event>
   <event>
diff --git a/asterix-events/src/main/resources/events/execute.sh b/asterix-events/src/main/resources/events/execute.sh
new file mode 100644
index 0000000..30f6c2f
--- /dev/null
+++ b/asterix-events/src/main/resources/events/execute.sh
@@ -0,0 +1,27 @@
+USERNAME=$1
+if [ ! -d $MANAGIX_HOME/logs ];
+then
+   mkdir -p $MANAGIX_HOME/logs
+fi
+LOGDIR=$MANAGIX_HOME/logs
+if [ $DAEMON == "false" ]; then 
+  if [ -z $USERNAME ]
+  then
+    cmd_output=$(ssh $IP_LOCATION "$ENV $SCRIPT $ARGS" 2>&1 >/dev/null) 
+    echo "ssh $IP_LOCATION $ENV $SCRIPT $ARGS" >> $LOGDIR/execute.log
+    echo "$cmd_output"
+  else
+    echo "ssh -l $USERNAME $IP_LOCATION $ENV $SCRIPT $ARGS" >> $LOGDIR/execute.log
+    cmd_output=$(ssh -l $USERNAME $IP_LOCATION "$ENV $SCRIPT $ARGS" 2>&1 >/dev/null) 
+    echo "$cmd_output"
+  fi  
+else 
+  if [ -z $USERNAME ];
+  then
+     echo "ssh $IP_LOCATION $ENV $SCRIPT $ARGS &" >> $LOGDIR/execute.log
+     ssh $IP_LOCATION "$ENV $SCRIPT $ARGS" &
+  else
+     echo "ssh -l $USERNAME $IP_LOCATION $ENV $SCRIPT $ARGS &" >> $LOGDIR/execute.log
+     ssh -l $USERNAME $IP_LOCATION "$ENV $SCRIPT $ARGS" &
+  fi   
+fi
diff --git a/asterix-events/src/main/resources/events/file/create_file.sh b/asterix-events/src/main/resources/events/file/create_file.sh
new file mode 100644
index 0000000..762a2d3
--- /dev/null
+++ b/asterix-events/src/main/resources/events/file/create_file.sh
@@ -0,0 +1,3 @@
+mkdir -p $1
+echo "touch $1/$2" >> ~/file_create.log
+touch $1/$2
diff --git a/asterix-events/src/main/resources/events/file/delete.sh b/asterix-events/src/main/resources/events/file/delete.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/file/dir_transfer.sh b/asterix-events/src/main/resources/events/file/dir_transfer.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/file/transfer.sh b/asterix-events/src/main/resources/events/file/transfer.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/hdfs/delete.sh b/asterix-events/src/main/resources/events/hdfs/delete.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_failure/nc_failure.sh b/asterix-events/src/main/resources/events/node_failure/nc_failure.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_info/node_info.sh b/asterix-events/src/main/resources/events/node_info/node_info.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_join/nc_join.sh b/asterix-events/src/main/resources/events/node_join/nc_join.sh
old mode 100755
new mode 100644
index e0254c9..a18fe09
--- a/asterix-events/src/main/resources/events/node_join/nc_join.sh
+++ b/asterix-events/src/main/resources/events/node_join/nc_join.sh
@@ -19,5 +19,7 @@
 then 
   mkdir -p $LOG_DIR
 fi
+
 cd $WORKING_DIR
+
 $ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port $CLUSTER_NET_PORT  -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -iodevices $IO_DEVICES -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
diff --git a/asterix-events/src/main/resources/events/node_restart/nc_restart.sh b/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql b/asterix-events/src/main/resources/events/prepare.sh
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
rename to asterix-events/src/main/resources/events/prepare.sh
diff --git a/asterix-events/src/main/resources/events/restore/restore.sh b/asterix-events/src/main/resources/events/restore/restore.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/schema/installer-conf.xsd b/asterix-events/src/main/resources/schema/installer-conf.xsd
similarity index 100%
rename from asterix-installer/src/main/resources/schema/installer-conf.xsd
rename to asterix-events/src/main/resources/schema/installer-conf.xsd
diff --git a/asterix-events/src/main/resources/scripts/execute.sh b/asterix-events/src/main/resources/scripts/execute.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/scripts/prepare.sh b/asterix-events/src/main/resources/scripts/prepare.sh
old mode 100755
new mode 100644
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index f8d5ea2..8f58b0f 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -34,53 +30,62 @@
 				</configuration>
 			</plugin>
 			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>appassembler-maven-plugin</artifactId>
-				<version>1.0</version>
+				<groupId>org.jvnet.jaxb2.maven2</groupId>
+				<artifactId>maven-jaxb2-plugin</artifactId>
 				<executions>
 					<execution>
-						<configuration>
-							<programs>
-								<program>
-									<mainClass>edu.uci.ics.asterix.drivers.AsterixWebServer</mainClass>
-									<name>asterix-web</name>
-								</program>
-								<program>
-									<mainClass>edu.uci.ics.asterix.drivers.AsterixClientDriver</mainClass>
-									<name>asterix-cmd</name>
-								</program>
-							</programs>
-							<repositoryLayout>flat</repositoryLayout>
-							<repositoryName>lib</repositoryName>
-						</configuration>
-						<phase>package</phase>
+						<id>configuration</id>
 						<goals>
-							<goal>assemble</goal>
+							<goal>generate</goal>
 						</goals>
+						<configuration>
+							<schemaDirectory>src/main/resources/schema</schemaDirectory>
+							<schemaIncludes>
+								<include>library.xsd</include>
+							</schemaIncludes>
+							<generatePackage>edu.uci.ics.asterix.external.library</generatePackage>
+							<generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
+						</configuration>
 					</execution>
 				</executions>
 			</plugin>
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<version>2.7.2</version>
+				<artifactId>maven-jar-plugin</artifactId>
+				<version>2.2</version>
 				<configuration>
-					<!-- doesn't work from m2eclipse, currently <additionalClasspathElements> 
-						<additionalClasspathElement>${basedir}/src/main/resources</additionalClasspathElement> 
-						</additionalClasspathElements> -->
-					<forkMode>pertest</forkMode>
-					<argLine>-enableassertions -Xmx${test.heap.size}m
-						-Dfile.encoding=UTF-8
-						-Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
 					<includes>
-						<include>**/*TestSuite.java</include>
-						<include>**/*Test.java</include>
+						<include>**/*.class</include>
+						<include>**/*.txt</include>
 					</includes>
 				</configuration>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+						<phase>package</phase>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<version>2.2-beta-5</version>
+				<executions>
+					<execution>
+						<configuration>
+							<descriptor>src/main/assembly/binary-assembly-libzip.xml</descriptor>
+							<finalName>testlib-zip</finalName>
+						</configuration>
+						<phase>package</phase>
+						<goals>
+							<goal>attached</goal>
+						</goals>
+					</execution>
+				</executions>
 			</plugin>
 		</plugins>
 	</build>
-
 	<dependencies>
 		<dependency>
 			<groupId>javax.servlet</groupId>
@@ -108,6 +113,17 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-metadata</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>algebricks-compiler</artifactId>
+		</dependency>
+		<dependency>
 			<groupId>com.kenai.nbpwr</groupId>
 			<artifactId>org-apache-commons-io</artifactId>
 			<version>1.3.1-201002241208</version>
@@ -154,5 +170,4 @@
 			<version>1.0</version>
 		</dependency>
 	</dependencies>
-
-</project>
+</project> 
diff --git a/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml b/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml
new file mode 100644
index 0000000..d76174a
--- /dev/null
+++ b/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml
@@ -0,0 +1,19 @@
+<assembly>
+  <id>binary-assembly</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>target/test-classes</directory>
+      <outputDirectory></outputDirectory>
+      <includes>
+        <include>**</include>
+      </includes>
+      <excludes>
+        <exclude>**.xml</exclude>
+      </excludes>
+    </fileSet>
+  </fileSets>
+</assembly>
diff --git a/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml b/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml
new file mode 100644
index 0000000..bec6e32
--- /dev/null
+++ b/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml
@@ -0,0 +1,23 @@
+<assembly>
+  <id>binary-assembly</id>
+  <formats>
+    <format>zip</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>target</directory>
+      <outputDirectory></outputDirectory>
+      <includes>
+        <include>*test*.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>src/test/resources</directory>
+      <outputDirectory></outputDirectory>
+      <includes>
+        <include>*.xml</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+</assembly>
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
index f1a3d6c..6880308 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
@@ -14,21 +14,74 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
- * A factory class for creating the @see {CNNFeedAdapter}.  
+ * A factory class for creating the @see {CNNFeedAdapter}.
  */
-public class CNNFeedAdapterFactory implements ITypedDatasetAdapterFactory {
+public class CNNFeedAdapterFactory implements ITypedAdapterFactory {
     private static final long serialVersionUID = 1L;
 
+    private Map<String, String> configuration;
+
+    private List<String> feedURLs = new ArrayList<String>();
+    private static Map<String, String> topicFeeds = new HashMap<String, String>();
+    private ARecordType recordType;
+
+    public static final String KEY_RSS_URL = "topic";
+    public static final String KEY_INTERVAL = "interval";
+    public static final String TOP_STORIES = "topstories";
+    public static final String WORLD = "world";
+    public static final String US = "us";
+    public static final String SPORTS = "sports";
+    public static final String BUSINESS = "business";
+    public static final String POLITICS = "politics";
+    public static final String CRIME = "crime";
+    public static final String TECHNOLOGY = "technology";
+    public static final String HEALTH = "health";
+    public static final String ENTERNTAINMENT = "entertainemnt";
+    public static final String TRAVEL = "travel";
+    public static final String LIVING = "living";
+    public static final String VIDEO = "video";
+    public static final String STUDENT = "student";
+    public static final String POPULAR = "popular";
+    public static final String RECENT = "recent";
+
+    private void initTopics() {
+        topicFeeds.put(TOP_STORIES, "http://rss.cnn.com/rss/cnn_topstories.rss");
+        topicFeeds.put(WORLD, "http://rss.cnn.com/rss/cnn_world.rss");
+        topicFeeds.put(US, "http://rss.cnn.com/rss/cnn_us.rss");
+        topicFeeds.put(SPORTS, "http://rss.cnn.com/rss/si_topstories.rss");
+        topicFeeds.put(BUSINESS, "http://rss.cnn.com/rss/money_latest.rss");
+        topicFeeds.put(POLITICS, "http://rss.cnn.com/rss/cnn_allpolitics.rss");
+        topicFeeds.put(CRIME, "http://rss.cnn.com/rss/cnn_crime.rss");
+        topicFeeds.put(TECHNOLOGY, "http://rss.cnn.com/rss/cnn_tech.rss");
+        topicFeeds.put(HEALTH, "http://rss.cnn.com/rss/cnn_health.rss");
+        topicFeeds.put(ENTERNTAINMENT, "http://rss.cnn.com/rss/cnn_showbiz.rss");
+        topicFeeds.put(LIVING, "http://rss.cnn.com/rss/cnn_living.rss");
+        topicFeeds.put(VIDEO, "http://rss.cnn.com/rss/cnn_freevideo.rss");
+        topicFeeds.put(TRAVEL, "http://rss.cnn.com/rss/cnn_travel.rss");
+        topicFeeds.put(STUDENT, "http://rss.cnn.com/rss/cnn_studentnews.rss");
+        topicFeeds.put(POPULAR, "http://rss.cnn.com/rss/cnn_mostpopular.rss");
+        topicFeeds.put(RECENT, "http://rss.cnn.com/rss/cnn_latest.rss");
+    }
+
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
-        CNNFeedAdapter cnnFeedAdapter = new CNNFeedAdapter();
-        cnnFeedAdapter.configure(configuration);
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        CNNFeedAdapter cnnFeedAdapter = new CNNFeedAdapter(configuration, recordType, ctx);
         return cnnFeedAdapter;
     }
 
@@ -37,4 +90,61 @@
         return "cnn_feed";
     }
 
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.TYPED;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+        if (rssURLProperty == null) {
+            throw new IllegalArgumentException("no rss url provided");
+        }
+        initializeFeedURLs(rssURLProperty);
+        recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
+                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+                false);
+
+    }
+
+    private void initializeFeedURLs(String rssURLProperty) {
+        feedURLs.clear();
+        String[] rssTopics = rssURLProperty.split(",");
+        initTopics();
+        for (String topic : rssTopics) {
+            String feedURL = topicFeeds.get(topic);
+            if (feedURL == null) {
+                throw new IllegalArgumentException(" unknown topic :" + topic + " please choose from the following "
+                        + getValidTopics());
+            }
+            feedURLs.add(feedURL);
+        }
+    }
+
+    private static String getValidTopics() {
+        StringBuilder builder = new StringBuilder();
+        for (String key : topicFeeds.keySet()) {
+            builder.append(key);
+            builder.append(" ");
+        }
+        return new String(builder);
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(feedURLs.size());
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return recordType;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index 4ca3d72..af057c9 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -22,10 +23,18 @@
 import org.apache.hadoop.mapred.JobConf;
 
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -34,7 +43,7 @@
  * A factory class for creating an instance of HDFSAdapter
  */
 @SuppressWarnings("deprecation")
-public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
+public class HDFSAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
     private static final long serialVersionUID = 1L;
 
     public static final String HDFS_ADAPTER_NAME = "hdfs";
@@ -52,7 +61,22 @@
     private boolean executed[];
     private InputSplitsFactory inputSplitsFactory;
     private ConfFactory confFactory;
-    private boolean setup = false;
+    private IAType atype;
+    private boolean configured = false;
+    public static Scheduler hdfsScheduler;
+    private static boolean initialized = false;
+
+    private static Scheduler initializeHDFSScheduler() {
+        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+        Scheduler scheduler = null;
+        try {
+            scheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
+                    .getClusterControllerInfo().getClientNetPort());
+        } catch (HyracksException e) {
+            throw new IllegalStateException("Cannot obtain hdfs scheduler");
+        }
+        return scheduler;
+    }
 
     private static final Map<String, String> formatClassNames = initInputFormatMap();
 
@@ -64,30 +88,12 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        if (!setup) {
-            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-            configureJobConf(configuration);
-            JobConf conf = configureJobConf(configuration);
-            confFactory = new ConfFactory(conf);
-
-            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
-            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-            inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
-            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-            readSchedule = scheduler.getLocationConstraints(inputSplits);
-            executed = new boolean[readSchedule.length];
-            Arrays.fill(executed, false);
-
-            setup = true;
-        }
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
         JobConf conf = confFactory.getConf();
         InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-        hdfsAdapter.configure(configuration);
+        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName,
+                parserFactory, ctx);
         return hdfsAdapter;
     }
 
@@ -96,7 +102,7 @@
         return HDFS_ADAPTER_NAME;
     }
 
-    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+    private JobConf configureJobConf(Map<String, String> configuration) throws Exception {
         JobConf conf = new JobConf();
         conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
         conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
@@ -107,4 +113,64 @@
         return conf;
     }
 
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.GENERIC;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        if (!configured) {
+            throw new IllegalStateException("Adapter factory has not been configured yet");
+        }
+        return (AlgebricksPartitionConstraint) clusterLocations;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+        if (!initialized) {
+            hdfsScheduler = initializeHDFSScheduler();
+            initialized = true;
+        }
+        this.configuration = configuration;
+        JobConf conf = configureJobConf(configuration);
+        confFactory = new ConfFactory(conf);
+
+        clusterLocations = getClusterLocations();
+        int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+        InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+        inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+        readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
+        executed = new boolean[readSchedule.length];
+        Arrays.fill(executed, false);
+        configured = true;
+
+        atype = (IAType) outputType;
+        configureFormat(atype);
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    private static AlgebricksPartitionConstraint getClusterLocations() {
+        ArrayList<String> locs = new ArrayList<String>();
+        Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+        for (String i : stores.keySet()) {
+            String[] nodeStores = stores.get(i);
+            int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+            for (int j = 0; j < nodeStores.length; j++) {
+                for (int k = 0; k < numIODevices; k++) {
+                    locs.add(i);
+                }
+            }
+        }
+        String[] cluster = new String[locs.size()];
+        cluster = locs.toArray(cluster);
+        return new AlgebricksAbsolutePartitionConstraint(cluster);
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index 409eb7a..991dadb 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -23,10 +23,14 @@
 
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -35,7 +39,7 @@
  * A factory class for creating an instance of HiveAdapter
  */
 @SuppressWarnings("deprecation")
-public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
+public class HiveAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
     private static final long serialVersionUID = 1L;
 
     public static final String HDFS_ADAPTER_NAME = "hdfs";
@@ -65,7 +69,8 @@
     private InputSplitsFactory inputSplitsFactory;
     private ConfFactory confFactory;
     private transient AlgebricksPartitionConstraint clusterLocations;
-    private boolean setup = false;
+    private boolean configured = false;
+    private IAType atype;
 
     private static final Map<String, String> formatClassNames = initInputFormatMap();
 
@@ -77,30 +82,12 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        if (!setup) {
-            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-            configureJobConf(configuration);
-            JobConf conf = configureJobConf(configuration);
-            confFactory = new ConfFactory(conf);
-
-            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
-            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-            inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
-            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-            readSchedule = scheduler.getLocationConstraints(inputSplits);
-            executed = new boolean[readSchedule.length];
-            Arrays.fill(executed, false);
-
-            setup = true;
-        }
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
         JobConf conf = confFactory.getConf();
         InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-        hiveAdapter.configure(configuration);
+        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+        HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations,
+                nodeName, parserFactory, ctx);
         return hiveAdapter;
     }
 
@@ -109,7 +96,43 @@
         return "hive";
     }
 
-    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.GENERIC;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+        if (!configured) {
+            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+            configureJobConf(configuration);
+            JobConf conf = configureJobConf(configuration);
+            confFactory = new ConfFactory(conf);
+
+            clusterLocations = AsterixClusterProperties.INSTANCE.getClusterLocations();
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+            Scheduler scheduler = HDFSAdapterFactory.hdfsScheduler;
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
+
+            atype = (IAType) outputType;
+            configureFormat(atype);
+            configured = true;
+        }
+
+    }
+
+    private JobConf configureJobConf(Map<String, String> configuration) throws Exception {
         JobConf conf = new JobConf();
 
         /** configure hive */
@@ -121,7 +144,7 @@
             tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
                     + configuration.get(HIVE_TABLE);
         }
-        configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+        configuration.put(HDFSAdapterFactory.KEY_PATH, tablePath);
         if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
             throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
         }
@@ -141,4 +164,11 @@
                 (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
         return conf;
     }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
deleted file mode 100644
index 5cb6777..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
+++ /dev/null
@@ -1,32 +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.external.adapter.factory;
-
-import java.io.Serializable;
-
-/**
- * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
- * Acts as a marker interface indicating that the implementation provides functionality
- * for creating an adapter.
- */
-public interface IAdapterFactory extends Serializable {
-
-    /**
-     * Returns the display name corresponding to the Adapter type that is created by the factory.
-     * 
-     * @return the display name
-     */
-    public String getName();
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
deleted file mode 100644
index 0a178a7..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ /dev/null
@@ -1,43 +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.external.adapter.factory;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-
-/**
- * A base interface for an adapter factory that creates instance of an adapter kind that
- * is 'generic' in nature. A 'typed' adapter returns records with a configurable datatype.
- */
-public interface IGenericDatasetAdapterFactory extends IAdapterFactory {
-
-    public static final String KEY_TYPE_NAME = "output-type-name";
-
-    /**
-     * Creates an instance of IDatasourceAdapter.
-     * 
-     * @param configuration
-     *            The configuration parameters for the adapter that is instantiated.
-     *            The passed-in configuration is used to configure the created instance of the adapter.
-     * @param atype
-     *            The type for the ADM records that are returned by the adapter.
-     * @return An instance of IDatasourceAdapter.
-     * @throws Exception
-     */
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
deleted file mode 100644
index 674bf52..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
+++ /dev/null
@@ -1,38 +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.external.adapter.factory;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-
-/**
- * A base interface for an adapter factory that creates instance of an adapter kind that
- * is 'typed' in nature. A 'typed' adapter returns records with a pre-defined datatype.
- */
-public interface ITypedDatasetAdapterFactory extends IAdapterFactory {
-
-    /**
-     * Creates an instance of IDatasourceAdapter.
-     * 
-     * @param configuration
-     *            The configuration parameters for the adapter that is instantiated.
-     *            The passed-in configuration is used to configure the created instance of the adapter.
-     * @return An instance of IDatasourceAdapter.
-     * @throws Exception
-     */
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception;
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index e680232..7c7b2a3 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -14,25 +14,43 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.io.File;
 import java.util.Map;
+import java.util.logging.Level;
 
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter;
+import edu.uci.ics.asterix.external.util.DNSResolverFactory;
+import edu.uci.ics.asterix.external.util.INodeResolver;
+import edu.uci.ics.asterix.external.util.INodeResolverFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 
 /**
  * Factory class for creating an instance of NCFileSystemAdapter. An
  * NCFileSystemAdapter reads external data residing on the local file system of
  * an NC.
  */
-public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
+public class NCFileSystemAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
     private static final long serialVersionUID = 1L;
+
     public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
 
+    private static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
+
+    private IAType sourceDatatype;
+    private FileSplit[] fileSplits;
+
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
-        fsAdapter.configure(configuration);
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(fileSplits, parserFactory, sourceDatatype, ctx);
         return fsAdapter;
     }
 
@@ -40,4 +58,88 @@
     public String getName() {
         return NC_FILE_SYSTEM_ADAPTER_NAME;
     }
+
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.GENERIC;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+        this.configuration = configuration;
+        String[] splits = ((String) configuration.get(KEY_PATH)).split(",");
+        IAType sourceDatatype = (IAType) outputType;
+        configureFileSplits(splits);
+        configureFormat(sourceDatatype);
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return configurePartitionConstraint();
+    }
+
+    private void configureFileSplits(String[] splits) throws AsterixException {
+        if (fileSplits == null) {
+            fileSplits = new FileSplit[splits.length];
+            String nodeName;
+            String nodeLocalPath;
+            int count = 0;
+            String trimmedValue;
+            for (String splitPath : splits) {
+                trimmedValue = splitPath.trim();
+                if (!trimmedValue.contains("://")) {
+                    throw new AsterixException("Invalid path: " + splitPath
+                            + "\nUsage- path=\"Host://Absolute File Path\"");
+                }
+                nodeName = trimmedValue.split(":")[0];
+                nodeLocalPath = trimmedValue.split("://")[1];
+                FileSplit fileSplit = new FileSplit(nodeName, new FileReference(new File(nodeLocalPath)));
+                fileSplits[count++] = fileSplit;
+            }
+        }
+    }
+
+    private AlgebricksPartitionConstraint configurePartitionConstraint() throws AsterixException {
+        String[] locs = new String[fileSplits.length];
+        String location;
+        for (int i = 0; i < fileSplits.length; i++) {
+            location = getNodeResolver().resolveNode(fileSplits[i].getNodeName());
+            locs[i] = location;
+        }
+        return new AlgebricksAbsolutePartitionConstraint(locs);
+    }
+
+    protected INodeResolver getNodeResolver() {
+        if (nodeResolver == null) {
+            nodeResolver = initNodeResolver();
+        }
+        return nodeResolver;
+    }
+
+    private static INodeResolver initNodeResolver() {
+        INodeResolver nodeResolver = null;
+        String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
+        if (configuredNodeResolverFactory != null) {
+            try {
+                nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
+                        .createNodeResolver();
+
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.log(Level.WARNING, "Unable to create node resolver from the configured classname "
+                            + configuredNodeResolverFactory + "\n" + e.getMessage());
+                }
+                nodeResolver = DEFAULT_NODE_RESOLVER;
+            }
+        } else {
+            nodeResolver = DEFAULT_NODE_RESOLVER;
+        }
+        return nodeResolver;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
index 232d5e8..9c137e1 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
@@ -16,23 +16,31 @@
 
 import java.util.Map;
 
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.PullBasedTwitterAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * Factory class for creating an instance of PullBasedTwitterAdapter.
  * This adapter provides the functionality of fetching tweets from Twitter service
  * via pull-based Twitter API.
  */
-public class PullBasedTwitterAdapterFactory implements ITypedDatasetAdapterFactory {
+public class PullBasedTwitterAdapterFactory implements ITypedAdapterFactory {
     private static final long serialVersionUID = 1L;
     public static final String PULL_BASED_TWITTER_ADAPTER_NAME = "pull_twitter";
 
+    private Map<String, String> configuration;
+    private static ARecordType recordType;
+
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
-        PullBasedTwitterAdapter twitterAdapter = new PullBasedTwitterAdapter();
-        twitterAdapter.configure(configuration);
-        return twitterAdapter;
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        return new PullBasedTwitterAdapter(configuration, ctx);
     }
 
     @Override
@@ -40,4 +48,39 @@
         return PULL_BASED_TWITTER_ADAPTER_NAME;
     }
 
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.TYPED;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        if (recordType != null) {
+            String[] fieldNames = { "id", "username", "location", "text", "timestamp" };
+            IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    BuiltinType.ASTRING };
+            try {
+                recordType = new ARecordType("FeedRecordType", fieldNames, fieldTypes, false);
+            } catch (Exception e) {
+                throw new IllegalStateException("Unable to create adapter output type");
+            }
+        }
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(1);
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return recordType;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
index ab18455..cc366f6 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
@@ -14,23 +14,39 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.RSSFeedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * Factory class for creating an instance of @see {RSSFeedAdapter}.
  * RSSFeedAdapter provides the functionality of fetching an RSS based feed.
  */
-public class RSSFeedAdapterFactory implements ITypedDatasetAdapterFactory {
+public class RSSFeedAdapterFactory implements ITypedAdapterFactory {
     private static final long serialVersionUID = 1L;
     public static final String RSS_FEED_ADAPTER_NAME = "rss_feed";
 
+    public static final String KEY_RSS_URL = "url";
+    public static final String KEY_INTERVAL = "interval";
+
+    private Map<String, String> configuration;
+    private ARecordType recordType;
+    private List<String> feedURLs = new ArrayList<String>();
+
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
-        RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter();
-        rssFeedAdapter.configure(configuration);
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter(configuration, recordType, ctx);
         return rssFeedAdapter;
     }
 
@@ -39,4 +55,51 @@
         return "rss_feed";
     }
 
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.TYPED;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+        if (rssURLProperty == null) {
+            throw new IllegalArgumentException("no rss url provided");
+        }
+        initializeFeedURLs(rssURLProperty);
+        configurePartitionConstraints();
+        recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
+                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+                false);
+
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(feedURLs.size());
+    }
+
+    private void initializeFeedURLs(String rssURLProperty) {
+        feedURLs.clear();
+        String[] feedURLProperty = rssURLProperty.split(",");
+        for (String feedURL : feedURLProperty) {
+            feedURLs.add(feedURL);
+        }
+    }
+
+    protected void configurePartitionConstraints() {
+
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return recordType;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
new file mode 100644
index 0000000..02ffdda
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
@@ -0,0 +1,120 @@
+package edu.uci.ics.asterix.external.adapter.factory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.util.DNSResolverFactory;
+import edu.uci.ics.asterix.external.util.INodeResolver;
+import edu.uci.ics.asterix.external.util.INodeResolverFactory;
+import edu.uci.ics.asterix.metadata.feeds.ConditionalPushTupleParserFactory;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public abstract class StreamBasedAdapterFactory implements IAdapterFactory {
+
+    private static final long serialVersionUID = 1L;
+    protected static final Logger LOGGER = Logger.getLogger(StreamBasedAdapterFactory.class.getName());
+
+    protected Map<String, String> configuration;
+    protected static INodeResolver nodeResolver;
+
+    public static final String KEY_FORMAT = "format";
+    public static final String KEY_PARSER_FACTORY = "parser";
+    public static final String KEY_DELIMITER = "delimiter";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_SOURCE_DATATYPE = "output-type-name";
+    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String FORMAT_ADM = "adm";
+    public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+    public static final String BATCH_SIZE = "batch-size";
+    public static final String BATCH_INTERVAL = "batch-interval";
+
+    protected ITupleParserFactory parserFactory;
+    protected ITupleParser parser;
+
+    protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
+    static {
+        typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
+        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+        typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
+        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+    }
+
+    protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+            throws AsterixException {
+        int n = recordType.getFieldTypes().length;
+        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+        for (int i = 0; i < n; i++) {
+            ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+            IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+            if (vpf == null) {
+                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+            }
+            fieldParserFactories[i] = vpf;
+        }
+        String delimiterValue = (String) configuration.get(KEY_DELIMITER);
+        if (delimiterValue != null && delimiterValue.length() > 1) {
+            throw new AsterixException("improper delimiter");
+        }
+
+        Character delimiter = delimiterValue.charAt(0);
+
+        return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, fieldParserFactories, delimiter,
+                configuration) : new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
+    }
+
+    protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+            throws AsterixException {
+        try {
+            return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, configuration)
+                    : new AdmSchemafullRecordParserFactory(recordType);
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+
+    }
+
+    protected void configureFormat(IAType sourceDatatype) throws Exception {
+        String propValue = (String) configuration.get(BATCH_SIZE);
+        int batchSize = propValue != null ? Integer.parseInt(propValue) : -1;
+        propValue = (String) configuration.get(BATCH_INTERVAL);
+        long batchInterval = propValue != null ? Long.parseLong(propValue) : -1;
+        boolean conditionalPush = batchSize > 0 || batchInterval > 0;
+
+        String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
+        if (parserFactoryClassname == null) {
+            String specifiedFormat = (String) configuration.get(KEY_FORMAT);
+            if (specifiedFormat == null) {
+                throw new IllegalArgumentException(" Unspecified data format");
+            } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
+                parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+            } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
+                parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+            } else {
+                throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
+            }
+        } else {
+            parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
+        }
+
+    }
+
+ 
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
deleted file mode 100644
index a48cfb8..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
+++ /dev/null
@@ -1,80 +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.external.data.operator;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-
-/**
- * Operator responsible for ingesting data from an external source. This
- * operator uses a (configurable) adapter associated with the feed dataset.
- */
-public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    private final String adapterFactoryClassName;
-    private final Map<String, Object> adapterConfiguration;
-    private final IAType atype;
-    private final FeedId feedId;
-    private final IAdapterFactory datasourceAdapterFactory;
-
-    public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedId feedId, String adapter,
-            Map<String, Object> arguments, ARecordType atype, RecordDescriptor rDesc,
-            IAdapterFactory datasourceAdapterFactory) {
-        super(spec, 1, 1);
-        recordDescriptors[0] = rDesc;
-        this.adapterFactoryClassName = adapter;
-        this.adapterConfiguration = arguments;
-        this.atype = atype;
-        this.feedId = feedId;
-        this.datasourceAdapterFactory = datasourceAdapterFactory;
-    }
-
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
-            throws HyracksDataException {
-        ITypedDatasourceAdapter adapter;
-        try {
-            if (datasourceAdapterFactory instanceof IGenericDatasetAdapterFactory) {
-                adapter = (ITypedDatasourceAdapter) ((IGenericDatasetAdapterFactory) datasourceAdapterFactory)
-                        .createAdapter(adapterConfiguration, atype);
-            } else if (datasourceAdapterFactory instanceof ITypedDatasetAdapterFactory) {
-                adapter = (ITypedDatasourceAdapter) ((ITypedDatasetAdapterFactory) datasourceAdapterFactory)
-                        .createAdapter(adapterConfiguration);
-            } else {
-                throw new IllegalStateException(" Unknown adapter factory type for " + adapterFactoryClassName);
-            }
-            adapter.initialize(ctx);
-        } catch (Exception e) {
-            throw new HyracksDataException("initialization of adapter failed", e);
-        }
-        return new FeedIntakeOperatorNodePushable(feedId, adapter, partition);
-    }
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
deleted file mode 100644
index 31470f3..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
+++ /dev/null
@@ -1,125 +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.external.data.operator;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.AlterFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-
-/**
- * The runtime for @see{FeedIntakeOperationDescriptor}
- */
-public class FeedIntakeOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-
-    private final IDatasourceAdapter adapter;
-    private final int partition;
-    private final IFeedManager feedManager;
-    private final FeedId feedId;
-    private final LinkedBlockingQueue<IFeedMessage> inbox;
-    private FeedInboxMonitor feedInboxMonitor;
-
-    public FeedIntakeOperatorNodePushable(FeedId feedId, IDatasourceAdapter adapter, int partition) {
-        this.adapter = adapter;
-        this.partition = partition;
-        this.feedManager = (IFeedManager) FeedManager.INSTANCE;
-        this.feedId = feedId;
-        inbox = new LinkedBlockingQueue<IFeedMessage>();
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        if (adapter instanceof IManagedFeedAdapter) {
-            feedInboxMonitor = new FeedInboxMonitor((IManagedFeedAdapter) adapter, inbox, partition);
-            AsterixThreadExecutor.INSTANCE.execute(feedInboxMonitor);
-            feedManager.registerFeedMsgQueue(feedId, inbox);
-        }
-        writer.open();
-        try {
-            adapter.start(partition, writer);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new HyracksDataException(e);
-            /* 
-             we do not throw an exception, but allow the operator to close
-             gracefully throwing an exception here would result in a job abort and a
-             transaction roll back that undoes all the work done so far.
-             */
-
-        } finally {
-            writer.close();
-            if (adapter instanceof IManagedFeedAdapter) {
-                feedManager.unregisterFeedMsgQueue(feedId, inbox);
-            }
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.close();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        // do nothing
-    }
-}
-
-class FeedInboxMonitor extends Thread {
-
-    private LinkedBlockingQueue<IFeedMessage> inbox;
-    private final IManagedFeedAdapter adapter;
-
-    public FeedInboxMonitor(IManagedFeedAdapter adapter, LinkedBlockingQueue<IFeedMessage> inbox, int partition) {
-        this.inbox = inbox;
-        this.adapter = adapter;
-    }
-
-    @Override
-    public void run() {
-        while (true) {
-            try {
-                IFeedMessage feedMessage = inbox.take();
-                switch (feedMessage.getMessageType()) {
-                    case STOP:
-                        adapter.stop();
-                        break;
-                    case ALTER:
-                        adapter.alter(((AlterFeedMessage) feedMessage).getAlteredConfParams());
-                        break;
-                }
-            } catch (InterruptedException ie) {
-                break;
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java
deleted file mode 100644
index b6bacef..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java
+++ /dev/null
@@ -1,63 +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.external.data.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/**
- * Runtime for the @see{FeedMessageOperatorDescriptor}
- */
-public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-
-    private final FeedId feedId;
-    private final List<IFeedMessage> feedMessages;
-    private IFeedManager feedManager;
-
-    public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, List<IFeedMessage> feedMessages,
-            boolean applyToAll, int partition, int nPartitions) {
-        this.feedId = feedId;
-        if (applyToAll) {
-            this.feedMessages = feedMessages;
-        } else {
-            this.feedMessages = new ArrayList<IFeedMessage>();
-            feedMessages.add(feedMessages.get(partition));
-        }
-        feedManager = (IFeedManager) FeedManager.INSTANCE;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        try {
-            writer.open();
-            for (IFeedMessage feedMessage : feedMessages) {
-                feedManager.deliverMessage(feedId, feedMessage);
-            }
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        } finally {
-            writer.close();
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
deleted file mode 100644
index f96a030..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
+++ /dev/null
@@ -1,93 +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.external.dataset.adapter;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-
-/**
- * Represents the base class that is required to be extended by every
- * implementation of the IDatasourceAdapter interface.
- */
-public abstract class AbstractDatasourceAdapter implements IDatasourceAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    protected Map<String, Object> configuration;
-    protected transient AlgebricksPartitionConstraint partitionConstraint;
-    protected IAType atype;
-    protected IHyracksTaskContext ctx;
-    protected AdapterType adapterType;
-
-    protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
-    static {
-        typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-    }
-
-    protected static final Map<String, Object> formatToParserFactoryMap = initializeFormatParserFactoryMap();
-
-    public static final String KEY_FORMAT = "format";
-    public static final String KEY_PARSER_FACTORY = "parser";
-    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
-    public static final String FORMAT_ADM = "adm";
-
-    private static Map<String, Object> initializeFormatParserFactoryMap() {
-        Map<String, Object> map = new HashMap<String, Object>();
-        map.put(FORMAT_DELIMITED_TEXT, "edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory");
-        map.put(FORMAT_ADM, "edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory");
-        return map;
-    }
-
-    /**
-     * Get the partition constraint chosen by the adapter.
-     * An adapter may have preferences as to where it needs to be instantiated and used.
-     */
-    public abstract AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
-    /**
-     * Get the configured value from the adapter configuration parameters, corresponding to the an attribute.
-     * 
-     * @param attribute
-     *            The attribute whose value needs to be obtained.
-     */
-    public Object getAdapterProperty(String attribute) {
-        return configuration.get(attribute);
-    }
-
-    /**
-     * Get the adapter configuration parameters.
-     * 
-     * @return A Map<String,String> instance representing the adapter configuration.
-     */
-    public Map<String, Object> getConfiguration() {
-        return configuration;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
index 9112aae..017b511 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
@@ -14,94 +14,25 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * An Adapter that provides the functionality of fetching news feed from CNN service
  * The Adapter provides news feed as ADM records.
  */
-public class CNNFeedAdapter extends RSSFeedAdapter implements IDatasourceAdapter, IManagedFeedAdapter {
+public class CNNFeedAdapter extends RSSFeedAdapter implements IDatasourceAdapter, IFeedAdapter {
 
-    private static final long serialVersionUID = 2523303758114582251L;
-    private List<String> feedURLs = new ArrayList<String>();
-    private static Map<String, String> topicFeeds = new HashMap<String, String>();
+    private static final long serialVersionUID = 1L;
 
-    public static final String KEY_RSS_URL = "topic";
-    public static final String KEY_INTERVAL = "interval";
-    public static final String TOP_STORIES = "topstories";
-    public static final String WORLD = "world";
-    public static final String US = "us";
-    public static final String SPORTS = "sports";
-    public static final String BUSINESS = "business";
-    public static final String POLITICS = "politics";
-    public static final String CRIME = "crime";
-    public static final String TECHNOLOGY = "technology";
-    public static final String HEALTH = "health";
-    public static final String ENTERNTAINMENT = "entertainemnt";
-    public static final String TRAVEL = "travel";
-    public static final String LIVING = "living";
-    public static final String VIDEO = "video";
-    public static final String STUDENT = "student";
-    public static final String POPULAR = "popular";
-    public static final String RECENT = "recent";
-
-    private void initTopics() {
-        topicFeeds.put(TOP_STORIES, "http://rss.cnn.com/rss/cnn_topstories.rss");
-        topicFeeds.put(WORLD, "http://rss.cnn.com/rss/cnn_world.rss");
-        topicFeeds.put(US, "http://rss.cnn.com/rss/cnn_us.rss");
-        topicFeeds.put(SPORTS, "http://rss.cnn.com/rss/si_topstories.rss");
-        topicFeeds.put(BUSINESS, "http://rss.cnn.com/rss/money_latest.rss");
-        topicFeeds.put(POLITICS, "http://rss.cnn.com/rss/cnn_allpolitics.rss");
-        topicFeeds.put(CRIME, "http://rss.cnn.com/rss/cnn_crime.rss");
-        topicFeeds.put(TECHNOLOGY, "http://rss.cnn.com/rss/cnn_tech.rss");
-        topicFeeds.put(HEALTH, "http://rss.cnn.com/rss/cnn_health.rss");
-        topicFeeds.put(ENTERNTAINMENT, "http://rss.cnn.com/rss/cnn_showbiz.rss");
-        topicFeeds.put(LIVING, "http://rss.cnn.com/rss/cnn_living.rss");
-        topicFeeds.put(VIDEO, "http://rss.cnn.com/rss/cnn_freevideo.rss");
-        topicFeeds.put(TRAVEL, "http://rss.cnn.com/rss/cnn_travel.rss");
-        topicFeeds.put(STUDENT, "http://rss.cnn.com/rss/cnn_studentnews.rss");
-        topicFeeds.put(POPULAR, "http://rss.cnn.com/rss/cnn_mostpopular.rss");
-        topicFeeds.put(RECENT, "http://rss.cnn.com/rss/cnn_latest.rss");
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        configuration = arguments;
-        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
-        if (rssURLProperty == null) {
-            throw new IllegalArgumentException("no rss url provided");
-        }
-        initializeFeedURLs(rssURLProperty);
-        configurePartitionConstraints();
-
-    }
-
-    private void initializeFeedURLs(String rssURLProperty) {
-        feedURLs.clear();
-        String[] rssTopics = rssURLProperty.split(",");
-        initTopics();
-        for (String topic : rssTopics) {
-            String feedURL = topicFeeds.get(topic);
-            if (feedURL == null) {
-                throw new IllegalArgumentException(" unknown topic :" + topic + " please choose from the following "
-                        + getValidTopics());
-            }
-            feedURLs.add(feedURL);
-        }
-    }
-
-    private static String getValidTopics() {
-        StringBuilder builder = new StringBuilder();
-        for (String key : topicFeeds.keySet()) {
-            builder.append(key);
-            builder.append(" ");
-        }
-        return new String(builder);
+    public CNNFeedAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
+            throws AsterixException {
+        super(configuration, recordType, ctx);
     }
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
index 33ee11f..45a827c 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
@@ -16,158 +16,37 @@
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.List;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.util.DNSResolverFactory;
-import edu.uci.ics.asterix.external.util.INodeResolver;
-import edu.uci.ics.asterix.external.util.INodeResolverFactory;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public abstract class FileSystemBasedAdapter extends AbstractDatasourceAdapter {
+public abstract class FileSystemBasedAdapter implements IDatasourceAdapter {
 
     private static final long serialVersionUID = 1L;
 
     public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
-    public static final String KEY_DELIMITER = "delimiter";
-    public static final String KEY_PATH = "path";
-
-    protected ITupleParserFactory parserFactory;
-    protected ITupleParser parser;
-    protected static INodeResolver nodeResolver;
-
-    private static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
-    private static final Logger LOGGER = Logger.getLogger(FileSystemBasedAdapter.class.getName());
 
     public abstract InputStream getInputStream(int partition) throws IOException;
 
-    public FileSystemBasedAdapter(IAType atype) {
-        this.atype = atype;
+    protected final ITupleParser tupleParser;
+    protected final IAType sourceDatatype;
+    protected IHyracksTaskContext ctx;
+
+    public FileSystemBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx) throws HyracksDataException {
+        this.tupleParser = parserFactory.createTupleParser(ctx);
+        this.sourceDatatype = sourceDatatype;
+        this.ctx = ctx;
     }
 
     @Override
     public void start(int partition, IFrameWriter writer) throws Exception {
         InputStream in = getInputStream(partition);
-        parser = getTupleParser();
-        parser.parse(in, writer);
+        tupleParser.parse(in, writer);
     }
 
-    @Override
-    public abstract void initialize(IHyracksTaskContext ctx) throws Exception;
-
-    @Override
-    public abstract void configure(Map<String, Object> arguments) throws Exception;
-
-    @Override
-    public abstract AdapterType getAdapterType();
-
-    @Override
-    public abstract AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
-    protected ITupleParser getTupleParser() throws Exception {
-        return parserFactory.createTupleParser(ctx);
-    }
-
-    protected void configureFormat() throws Exception {
-        String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
-        if (parserFactoryClassname == null) {
-            String specifiedFormat = (String) configuration.get(KEY_FORMAT);
-            if (specifiedFormat == null) {
-                throw new IllegalArgumentException(" Unspecified data format");
-            } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
-                parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
-            } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
-                parserFactory = getADMDataTupleParserFactory((ARecordType) atype);
-            } else {
-                throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
-            }
-        } else {
-            parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
-        }
-
-    }
-
-    protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType) throws AsterixException {
-        int n = recordType.getFieldTypes().length;
-        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
-        for (int i = 0; i < n; i++) {
-            ATypeTag tag = null;
-            if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
-                List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
-                if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
-                    throw new NotImplementedException("Non-optional UNION type is not supported.");
-                }
-                tag = unionTypes.get(1).getTypeTag();
-            } else {
-                tag = recordType.getFieldTypes()[i].getTypeTag();
-            }
-            if (tag == null) {
-                throw new NotImplementedException("Failed to get the type information for field " + i + ".");
-            }
-            IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
-            if (vpf == null) {
-                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-            }
-            fieldParserFactories[i] = vpf;
-        }
-        String delimiterValue = (String) configuration.get(KEY_DELIMITER);
-        if (delimiterValue != null && delimiterValue.length() > 1) {
-            throw new AsterixException("improper delimiter");
-        }
-
-        Character delimiter = delimiterValue.charAt(0);
-        return new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
-    }
-
-    protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType) throws AsterixException {
-        try {
-            return new AdmSchemafullRecordParserFactory(recordType);
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-
-    }
-
-    protected INodeResolver getNodeResolver() {
-        if (nodeResolver == null) {
-            nodeResolver = initNodeResolver();
-        }
-        return nodeResolver;
-    }
-
-    private static INodeResolver initNodeResolver() {
-        INodeResolver nodeResolver = null;
-        String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
-        if (configuredNodeResolverFactory != null) {
-            try {
-                nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
-                        .createNodeResolver();
-
-            } catch (Exception e) {
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.log(Level.WARNING, "Unable to create node resolver from the configured classname "
-                            + configuredNodeResolverFactory + "\n" + e.getMessage());
-                }
-                nodeResolver = DEFAULT_NODE_RESOLVER;
-            }
-        } else {
-            nodeResolver = DEFAULT_NODE_RESOLVER;
-        }
-        return nodeResolver;
-    }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index f8b381b..1e694e2 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -16,7 +16,6 @@
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Map;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Counters.Counter;
@@ -31,6 +30,8 @@
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
  * Provides functionality for fetching external data stored in an HDFS instance.
@@ -44,34 +45,16 @@
     private transient boolean executed[];
     private transient InputSplit[] inputSplits;
     private transient JobConf conf;
-    private transient AlgebricksPartitionConstraint clusterLocations;
-
     private transient String nodeName;
 
     public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-            AlgebricksPartitionConstraint clusterLocations) {
-        super(atype);
+            String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx) throws HyracksDataException {
+        super(parserFactory, atype, ctx);
         this.readSchedule = readSchedule;
         this.executed = executed;
         this.inputSplits = inputSplits;
         this.conf = conf;
-        this.clusterLocations = clusterLocations;
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        this.configuration = arguments;
-        configureFormat();
-    }
-
-    public AdapterType getAdapterType() {
-        return AdapterType.READ_WRITE;
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        this.ctx = ctx;
-        this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+        this.nodeName = nodeName;
     }
 
     private Reporter getReporter() {
@@ -227,9 +210,4 @@
 
     }
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return clusterLocations;
-    }
-
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
index 91cf0d2..f4ff44e 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
@@ -14,21 +14,23 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.util.Map;
+import java.io.IOException;
+import java.io.InputStream;
 
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
  * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
  */
 @SuppressWarnings("deprecation")
-public class HiveAdapter extends AbstractDatasourceAdapter {
+public class HiveAdapter extends FileSystemBasedAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -42,35 +44,16 @@
     private HDFSAdapter hdfsAdapter;
 
     public HiveAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-            AlgebricksPartitionConstraint clusterLocations) {
-        this.hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-        this.atype = atype;
+            AlgebricksPartitionConstraint clusterLocations, String nodeName, ITupleParserFactory parserFactory,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        super(parserFactory, atype, ctx);
+        this.hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName, parserFactory,
+                ctx);
     }
 
     @Override
-    public AdapterType getAdapterType() {
-        return AdapterType.READ;
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        this.configuration = arguments;
-        this.hdfsAdapter.configure(arguments);
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        hdfsAdapter.initialize(ctx);
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        hdfsAdapter.start(partition, writer);
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return hdfsAdapter.getPartitionConstraint();
+    public InputStream getInputStream(int partition) throws IOException {
+        return hdfsAdapter.getInputStream(partition);
     }
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
deleted file mode 100644
index 9403bfe..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
+++ /dev/null
@@ -1,149 +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.external.dataset.adapter;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * A super interface implemented by a data source adapter. An adapter can be a
- * pull based or push based. This interface provides all common APIs that need
- * to be implemented by each adapter irrespective of the the kind of
- * adapter(pull or push).
- */
-public interface IDatasourceAdapter extends Serializable {
-
-    /**
-     * An adapter can be used to read from an external data source and may also
-     * allow writing to the external data source. This enum type indicates the
-     * kind of operations supported by the adapter.
-     * 
-     * @caller Compiler uses this method to assert the validity of an operation
-     *         on an external dataset. The type of adapter associated with an
-     *         external dataset determines the set of valid operations allowed
-     *         on the dataset.
-     */
-    public enum AdapterType {
-        READ,
-        WRITE,
-        READ_WRITE
-    }
-
-    /**
-     * Returns the type of adapter indicating if the adapter can be used for
-     * reading from an external data source or writing to an external data
-     * source or can be used for both purposes.
-     * 
-     * @Caller: Compiler: The compiler uses this API to verify if an operation
-     *          is supported by the adapter. For example, an write query against
-     *          an external dataset will not compile successfully if the
-     *          external dataset was declared with a read_only adapter.
-     * @see AdapterType
-     * @return
-     */
-    public AdapterType getAdapterType();
-
-    /**
-     * Each adapter instance is configured with a set of parameters that are
-     * key-value pairs. When creating an external or a feed dataset, an adapter
-     * instance is used in conjunction with a set of configuration parameters
-     * for the adapter instance. The configuration parameters are stored
-     * internally with the adapter and can be retrieved using this API.
-     * 
-     * @param propertyKey
-     * @return String the value corresponding to the configuration parameter
-     *         represented by the key- attributeKey.
-     */
-    public Object getAdapterProperty(String propertyKey);
-
-    /**
-     * Configures the IDatasourceAdapter instance.
-     * 
-     * @caller Scenario 1) Called during compilation of DDL statement that
-     *         creates a Feed dataset and associates the adapter with the
-     *         dataset. The (key,value) configuration parameters provided as
-     *         part of the DDL statement are collected by the compiler and
-     *         passed on to this method. The adapter may as part of
-     *         configuration connect with the external data source and determine
-     *         the IAType associated with data residing with the external
-     *         datasource.
-     *         Scenario 2) An adapter instance is created by an ASTERIX operator
-     *         that wraps around the adapter instance. The operator, as part of
-     *         its initialization invokes the configure method. The (key,value)
-     *         configuration parameters are passed on to the operator by the
-     *         compiler. Subsequent to the invocation, the wrapping operator
-     *         obtains the partition constraints (if any). In addition, in the
-     *         case of a read adapter, the wrapping operator obtains the output
-     *         ASTERIX type associated with the data that will be output from
-     *         the adapter.
-     * @param arguments
-     *            A map with key-value pairs that contains the configuration
-     *            parameters for the adapter. The arguments are obtained from
-     *            the metadata. Recall that the DDL to create an external
-     *            dataset or a feed dataset requires using an adapter and
-     *            providing all arguments as a set of (key,value) pairs. These
-     *            arguments are put into the metadata.
-     */
-    public void configure(Map<String, Object> arguments) throws Exception;
-
-    /**
-     * Returns a list of partition constraints. A partition constraint can be a
-     * requirement to execute at a particular location or could be cardinality
-     * constraints indicating the number of instances that need to run in
-     * parallel. example, a IDatasourceAdapter implementation written for data
-     * residing on the local file system of a node cannot run on any other node
-     * and thus has a location partition constraint. The location partition
-     * constraint can be expressed as a node IP address or a node controller id.
-     * In the former case, the IP address is translated to a node controller id
-     * running on the node with the given IP address.
-     * 
-     * @Caller The wrapper operator configures its partition constraints from
-     *         the constraints obtained from the adapter.
-     */
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
-    /**
-     * Allows the adapter to establish connection with the external data source
-     * expressing intent for data and providing any configuration parameters
-     * required by the external data source for the transfer of data. This
-     * method does not result in any data transfer, but is a prerequisite for
-     * any subsequent data transfer to happen between the external data source
-     * and the adapter.
-     * 
-     * @caller This method is called by the wrapping ASTERIX operator that
-     * @param ctx
-     * @throws Exception
-     */
-    public void initialize(IHyracksTaskContext ctx) throws Exception;
-
-    /**
-     * Triggers the adapter to begin ingestion of data from the external source.
-     * 
-     * @param partition
-     *            The adapter could be running with a degree of parallelism.
-     *            partition corresponds to the i'th parallel instance.
-     * @param writer
-     *            The instance of frame writer that is used by the adapter to
-     *            write frame to. Adapter packs the fetched bytes (from external source),
-     *            packs them into frames and forwards the frames to an upstream receiving
-     *            operator using the instance of IFrameWriter.
-     * @throws Exception
-     */
-    public void start(int partition, IFrameWriter writer) throws Exception;
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java
new file mode 100644
index 0000000..4fb14ff
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface IFeedClientFactory {
+
+    public IPullBasedFeedClient createFeedClient(IHyracksTaskContext ctx, Map<String, String> configuration)
+            throws Exception;
+
+    public ARecordType getRecordType() throws AsterixException;
+
+    public FeedClientType getFeedClientType();
+
+    public enum FeedClientType {
+        GENERIC,
+        TYPED
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
index aeecb5f..ee28c3a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
@@ -15,11 +15,18 @@
 package edu.uci.ics.asterix.external.dataset.adapter;
 
 import java.io.DataOutput;
+import java.util.Map;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 
 public interface IPullBasedFeedClient {
 
+    public enum InflowState {
+        NO_MORE_DATA,
+        DATA_AVAILABLE,
+        DATA_NOT_AVAILABLE
+    }
+
     /**
      * Writes the next fetched tuple into the provided instance of DatatOutput.
      * 
@@ -29,7 +36,7 @@
      *         false if no record was written to the DataOutput instance indicating non-availability of new data.
      * @throws AsterixException
      */
-    public boolean nextTuple(DataOutput dataOutput) throws AsterixException;
+    public InflowState nextTuple(DataOutput dataOutput) throws AsterixException;
 
     /**
      * Provides logic for any corrective action that feed client needs to execute on
@@ -42,10 +49,10 @@
     public void resetOnFailure(Exception e) throws AsterixException;
 
     /**
-     * Terminates a feed, that is data ingestion activity ceases.
-     * 
-     * @throws Exception
+     * @param configuration
      */
-    public void stop() throws Exception;
+    public boolean alter(Map<String, String> configuration);
+
+    public void stop();
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java
new file mode 100644
index 0000000..da49240
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java
@@ -0,0 +1,47 @@
+/*
+ * 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.external.dataset.adapter;
+
+import java.io.InputStream;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public interface IPushBasedFeedClient {
+
+    /**
+     * @return
+     * @throws AsterixException
+     */
+    public InputStream getInputStream() throws AsterixException;
+
+    /**
+     * Provides logic for any corrective action that feed client needs to execute on
+     * encountering an exception.
+     * 
+     * @param e
+     *            The exception encountered during fetching of data from external source
+     * @throws AsterixException
+     */
+    public void resetOnFailure(Exception e) throws AsterixException;
+
+    /**
+     * @param configuration
+     */
+    public boolean alter(Map<String, Object> configuration);
+
+    public void stop();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
index 182ccf7..ea8ed05 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
@@ -19,15 +19,12 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Map;
 
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
  * Factory class for creating an instance of NCFileSystemAdapter. An
@@ -37,59 +34,13 @@
 public class NCFileSystemAdapter extends FileSystemBasedAdapter {
 
     private static final long serialVersionUID = 1L;
-    private FileSplit[] fileSplits;
 
-    public NCFileSystemAdapter(IAType atype) {
-        super(atype);
-    }
+    private final FileSplit[] fileSplits;
 
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        this.configuration = arguments;
-        String[] splits = ((String) arguments.get(KEY_PATH)).split(",");
-        configureFileSplits(splits);
-        configureFormat();
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        this.ctx = ctx;
-    }
-
-    @Override
-    public AdapterType getAdapterType() {
-        return AdapterType.READ;
-    }
-
-    private void configureFileSplits(String[] splits) throws AsterixException {
-        if (fileSplits == null) {
-            fileSplits = new FileSplit[splits.length];
-            String nodeName;
-            String nodeLocalPath;
-            int count = 0;
-            String trimmedValue;
-            for (String splitPath : splits) {
-                trimmedValue = splitPath.trim();
-                if (!trimmedValue.contains("://")) {
-                    throw new AsterixException("Invalid path: " + splitPath
-                            + "\nUsage- path=\"Host://Absolute File Path\"");
-                }
-                nodeName = trimmedValue.split(":")[0];
-                nodeLocalPath = trimmedValue.split("://")[1];
-                FileSplit fileSplit = new FileSplit(nodeName, new FileReference(new File(nodeLocalPath)));
-                fileSplits[count++] = fileSplit;
-            }
-        }
-    }
-
-    private void configurePartitionConstraint() throws AsterixException {
-        String[] locs = new String[fileSplits.length];
-        String location;
-        for (int i = 0; i < fileSplits.length; i++) {
-            location = getNodeResolver().resolveNode(fileSplits[i].getNodeName());
-            locs[i] = location;
-        }
-        partitionConstraint = new AlgebricksAbsolutePartitionConstraint(locs);
+    public NCFileSystemAdapter(FileSplit[] fileSplits, ITupleParserFactory parserFactory, IAType atype,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        super(parserFactory, atype, ctx);
+        this.fileSplits = fileSplits;
     }
 
     @Override
@@ -105,11 +56,4 @@
         }
     }
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (partitionConstraint == null) {
-            configurePartitionConstraint();
-        }
-        return partitionConstraint;
-    }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
index 93d2057..da2264d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
@@ -15,10 +15,17 @@
 package edu.uci.ics.asterix.external.dataset.adapter;
 
 import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient.InflowState;
+import edu.uci.ics.asterix.metadata.feeds.AbstractFeedDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -29,18 +36,39 @@
  * Captures the common logic for obtaining bytes from an external source
  * and packing them into frames as tuples.
  */
-public abstract class PullBasedAdapter extends AbstractDatasourceAdapter implements ITypedDatasourceAdapter {
+public abstract class PullBasedAdapter extends AbstractFeedDatasourceAdapter implements IDatasourceAdapter,
+        IFeedAdapter {
 
     private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(PullBasedAdapter.class.getName());
 
     protected ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
     protected IPullBasedFeedClient pullBasedFeedClient;
     protected ARecordType adapterOutputType;
     private FrameTupleAppender appender;
     private ByteBuffer frame;
+    protected boolean continueIngestion = true;
+    protected boolean alterRequested = false;
+    private Map<String, String> modifiedConfiguration = null;
+    private long tupleCount = 0;
+    private final IHyracksTaskContext ctx;
+    protected Map<String, String> configuration;
 
     public abstract IPullBasedFeedClient getFeedClient(int partition) throws Exception;
 
+    public PullBasedAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) {
+        this.ctx = ctx;
+        this.configuration = configuration;
+    }
+
+    public long getIngestedRecordsCount() {
+        return tupleCount;
+    }
+
+    public void alter(Map<String, String> modifedConfiguration) {
+        this.modifiedConfiguration = modifedConfiguration;
+    }
+
     @Override
     public void start(int partition, IFrameWriter writer) throws Exception {
         appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -48,22 +76,45 @@
         appender.reset(frame, true);
 
         pullBasedFeedClient = getFeedClient(partition);
-        boolean moreData = false;
-        while (true) {
+        InflowState inflowState = null;
+        while (continueIngestion) {
             tupleBuilder.reset();
             try {
-                moreData = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput());
-                if (moreData) {
-                    tupleBuilder.addFieldEndOffset();
-                    appendTupleToFrame(writer);
-                } else {
-                    FrameUtils.flushFrame(frame, writer);
-                    break;
+                inflowState = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput());
+                switch (inflowState) {
+                    case DATA_AVAILABLE:
+                        tupleBuilder.addFieldEndOffset();
+                        appendTupleToFrame(writer);
+                        tupleCount++;
+                        break;
+                    case NO_MORE_DATA:
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Reached end of feed");
+                        }
+                        FrameUtils.flushFrame(frame, writer);
+                        continueIngestion = false;
+                        break;
+                    case DATA_NOT_AVAILABLE:
+                        break;
+                }
+                if (alterRequested) {
+                    boolean success = pullBasedFeedClient.alter(modifiedConfiguration);
+                    if (success) {
+                        configuration = modifiedConfiguration;
+                        modifiedConfiguration = null;
+                    }
                 }
             } catch (Exception failureException) {
                 try {
-                    pullBasedFeedClient.resetOnFailure(failureException);
-                    continue;
+                    failureException.printStackTrace();
+                    boolean continueIngestion = policyEnforcer.handleSoftwareFailure(failureException);
+                    if (continueIngestion) {
+                        pullBasedFeedClient.resetOnFailure(failureException);
+                        tupleBuilder.reset();
+                        continue;
+                    } else {
+                        throw failureException;
+                    }
                 } catch (Exception recoveryException) {
                     throw new Exception(recoveryException);
                 }
@@ -71,19 +122,11 @@
         }
     }
 
-    /**
-     * Allows an adapter to handle a runtime exception.
-     * @param e exception encountered during runtime
-     * @throws AsterixException
-     */
-    public void resetOnFailure(Exception e) throws AsterixException {
-        pullBasedFeedClient.resetOnFailure(e);
-        tupleBuilder.reset();
-    }
-
     private void appendTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+            System.out.println("flushing frame");
             FrameUtils.flushFrame(frame, writer);
+            System.out.println("flushed frame");
             appender.reset(frame, true);
             if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
                     tupleBuilder.getSize())) {
@@ -92,9 +135,17 @@
         }
     }
 
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return adapterOutputType;
+    /**
+     * Discontinue the ingestion of data and end the feed.
+     * 
+     * @throws Exception
+     */
+    public void stop() throws Exception {
+        continueIngestion = false;
+    }
+
+    public Map<String, String> getConfiguration() {
+        return configuration;
     }
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
index 6d05bde..8efe919 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
@@ -17,46 +17,138 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+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.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
 import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+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;
 
 public abstract class PullBasedFeedClient implements IPullBasedFeedClient {
 
     protected ARecordSerializerDeserializer recordSerDe;
     protected AMutableRecord mutableRecord;
     protected boolean messageReceived;
-    protected boolean continueIngestion=true;
+    protected boolean continueIngestion = true;
+    protected IARecordBuilder recordBuilder = new RecordBuilder();
 
-    public abstract boolean setNextRecord() throws Exception;
+    protected AMutableString aString = new AMutableString("");
+    protected AMutableInt32 aInt32 = new AMutableInt32(0);
+    protected AMutablePoint aPoint = new AMutablePoint(0, 0);
+    protected AMutableDateTime aDateTime = new AMutableDateTime(0);
+
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ASTRING);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+
+    public abstract InflowState setNextRecord() throws Exception;
 
     @Override
-    public boolean nextTuple(DataOutput dataOutput) throws AsterixException {
+    public InflowState nextTuple(DataOutput dataOutput) throws AsterixException {
         try {
-            boolean newData = setNextRecord();
-            if (newData && continueIngestion) {
-                IAType t = mutableRecord.getType();
-                ATypeTag tag = t.getTypeTag();
-                try {
+            System.out.println("Setting next record");
+            InflowState state = setNextRecord();
+            boolean first = true;
+            switch (state) {
+                case DATA_AVAILABLE:
+                    IAType t = mutableRecord.getType();
+                    ATypeTag tag = t.getTypeTag();
                     dataOutput.writeByte(tag.serialize());
-                } catch (IOException e) {
-                    throw new HyracksDataException(e);
-                }
-                recordSerDe.serialize(mutableRecord, dataOutput);
-                return true;
+                    if (first) {
+                        recordBuilder.reset(mutableRecord.getType());
+                        first = false;
+                    }
+                    recordBuilder.init();
+                    writeRecord(mutableRecord, dataOutput, recordBuilder);
+                    break;
+
+                case DATA_NOT_AVAILABLE:
+                    break;
+                case NO_MORE_DATA:
+                    break;
             }
-            return false;
+            return state;
         } catch (Exception e) {
             throw new AsterixException(e);
         }
 
     }
 
-    @Override
-    public void stop() {
-        continueIngestion = false;
+    private void writeRecord(AMutableRecord record, DataOutput dataOutput, IARecordBuilder recordBuilder)
+            throws IOException, AsterixException {
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        int numFields = record.getType().getFieldNames().length;
+        for (int pos = 0; pos < numFields; pos++) {
+            fieldValue.reset();
+            IAObject obj = record.getValueByPos(pos);
+            writeObject(obj, fieldValue.getDataOutput());
+            recordBuilder.addField(pos, fieldValue);
+        }
+        recordBuilder.write(dataOutput, false);
+    }
+
+    private void writeObject(IAObject obj, DataOutput dataOutput) throws IOException, AsterixException {
+        switch (obj.getType().getTypeTag()) {
+            case RECORD:
+                ATypeTag tag = obj.getType().getTypeTag();
+                try {
+                    dataOutput.writeByte(tag.serialize());
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                IARecordBuilder recordBuilder = new RecordBuilder();
+                recordBuilder.reset((ARecordType) obj.getType());
+                recordBuilder.init();
+                writeRecord((AMutableRecord) obj, dataOutput, recordBuilder);
+                break;
+            case UNORDEREDLIST:
+                tag = obj.getType().getTypeTag();
+                try {
+                    dataOutput.writeByte(tag.serialize());
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+                listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
+                IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
+                ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
+                while (cursor.next()) {
+                    listItemValue.reset();
+                    IAObject item = cursor.get();
+                    writeObject(item, listItemValue.getDataOutput());
+                    listBuilder.addItem(listItemValue);
+                }
+                listBuilder.write(dataOutput, false);
+                break;
+            default:
+                AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(obj.getType()).serialize(obj,
+                        dataOutput);
+                break;
+        }
     }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
index 2715a00..41143d8 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
@@ -14,32 +14,28 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * An adapter that provides the functionality of receiving tweets from the
  * Twitter service in the form of ADM formatted records.
  */
-public class PullBasedTwitterAdapter extends PullBasedAdapter implements IManagedFeedAdapter {
+public class PullBasedTwitterAdapter extends PullBasedAdapter implements IFeedAdapter {
 
     private static final long serialVersionUID = 1L;
 
     public static final String QUERY = "query";
     public static final String INTERVAL = "interval";
 
-    private boolean alterRequested = false;
-    private Map<String, String> alteredParams = new HashMap<String, String>();
     private ARecordType recordType;
-
+    private final IHyracksTaskContext ctx;
     private PullBasedTwitterFeedClient tweetClient;
 
     @Override
@@ -47,27 +43,13 @@
         return tweetClient;
     }
 
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        configuration = arguments;
-        String[] fieldNames = { "id", "username", "location", "text", "timestamp" };
-        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
-                BuiltinType.ASTRING };
-        recordType = new ARecordType("FeedRecordType", fieldNames, fieldTypes, false);
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
+    public PullBasedTwitterAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) throws AsterixException {
+        super(configuration, ctx);
         this.ctx = ctx;
         tweetClient = new PullBasedTwitterFeedClient(ctx, this);
     }
 
     @Override
-    public AdapterType getAdapterType() {
-        return AdapterType.READ;
-    }
-
-    @Override
     public void stop() {
         tweetClient.stop();
     }
@@ -75,33 +57,18 @@
     @Override
     public void alter(Map<String, String> properties) {
         alterRequested = true;
-        this.alteredParams = properties;
     }
 
     public boolean isAlterRequested() {
         return alterRequested;
     }
 
-    public Map<String, String> getAlteredParams() {
-        return alteredParams;
-    }
-
     public void postAlteration() {
-        alteredParams = null;
         alterRequested = false;
     }
 
-    @Override
     public ARecordType getAdapterOutputType() {
         return recordType;
     }
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (partitionConstraint == null) {
-            partitionConstraint = new AlgebricksCountPartitionConstraint(1);
-        }
-        return partitionConstraint;
-    }
-
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
index 9f44a03..7a5aeea 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
@@ -14,9 +14,10 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
-import java.util.Queue;
+import java.util.UUID;
+import java.util.logging.Logger;
 
 import twitter4j.Query;
 import twitter4j.QueryResult;
@@ -41,15 +42,17 @@
 
     private String keywords;
     private Query query;
-    private long id = 0;
     private String id_prefix;
     private Twitter twitter;
     private int requestInterval = 10; // seconds
-    private Queue<Tweet> tweetBuffer = new LinkedList<Tweet>();
+    private QueryResult result;
 
-    IAObject[] mutableFields;
-    String[] tupleFieldValues;
+    private IAObject[] mutableFields;
+    private String[] tupleFieldValues;
     private ARecordType recordType;
+    private int nextTweetIndex = 0;
+
+    private static final Logger LOGGER = Logger.getLogger(PullBasedTwitterFeedClient.class.getName());
 
     public PullBasedTwitterFeedClient(IHyracksTaskContext ctx, PullBasedTwitterAdapter adapter) {
         this.id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
@@ -59,24 +62,8 @@
         recordType = adapter.getAdapterOutputType();
         recordSerDe = new ARecordSerializerDeserializer(recordType);
         mutableRecord = new AMutableRecord(recordType, mutableFields);
-        initialize(adapter.getConfiguration());
         tupleFieldValues = new String[recordType.getFieldNames().length];
-    }
-
-    public void initialize(Map<String, Object> params) {
-        this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
-        this.query = new Query(keywords);
-        query.setRpp(100);
-    }
-
-    private Tweet getNextTweet() throws TwitterException, InterruptedException {
-        if (tweetBuffer.isEmpty()) {
-            QueryResult result;
-            Thread.sleep(1000 * requestInterval);
-            result = twitter.search(query);
-            tweetBuffer.addAll(result.getTweets());
-        }
-        return tweetBuffer.remove();
+        initialize(adapter.getConfiguration());
     }
 
     public ARecordType getRecordType() {
@@ -88,15 +75,14 @@
     }
 
     @Override
-    public boolean setNextRecord() throws Exception {
+    public InflowState setNextRecord() throws Exception {
         Tweet tweet;
         tweet = getNextTweet();
         if (tweet == null) {
-            return false;
+            return InflowState.DATA_NOT_AVAILABLE;
         }
         int numFields = recordType.getFieldNames().length;
-
-        tupleFieldValues[0] = id_prefix + ":" + id;
+        tupleFieldValues[0] = UUID.randomUUID().toString();
         tupleFieldValues[1] = tweet.getFromUser();
         tupleFieldValues[2] = tweet.getLocation() == null ? "" : tweet.getLocation();
         tupleFieldValues[3] = tweet.getText();
@@ -105,8 +91,7 @@
             ((AMutableString) mutableFields[i]).setValue(tupleFieldValues[i]);
             mutableRecord.setValueAtPos(i, mutableFields[i]);
         }
-        id++;
-        return true;
+        return InflowState.DATA_AVAILABLE;
     }
 
     @Override
@@ -114,4 +99,32 @@
         // TOOO: implement resetting logic for Twitter
     }
 
+    @Override
+    public boolean alter(Map<String, String> configuration) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public void stop() {
+        // TODO Auto-generated method stub
+    }
+
+    private void initialize(Map<String, String> params) {
+        this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
+        this.requestInterval = Integer.parseInt((String) params.get(PullBasedTwitterAdapter.INTERVAL));
+        this.query = new Query(keywords);
+        query.setRpp(100);
+    }
+
+    private Tweet getNextTweet() throws TwitterException, InterruptedException {
+        if (result == null || nextTweetIndex >= result.getTweets().size()) {
+            Thread.sleep(1000 * requestInterval);
+            result = twitter.search(query);
+            nextTweetIndex = 0;
+        }
+        List<Tweet> tw = result.getTweets();
+        return tw.get(nextTweetIndex++);
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
index 8379b18..102482d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
@@ -19,18 +19,15 @@
 import java.util.List;
 import java.util.Map;
 
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * RSSFeedAdapter provides the functionality of fetching an RSS based feed.
  */
-public class RSSFeedAdapter extends PullBasedAdapter implements IManagedFeedAdapter {
+public class RSSFeedAdapter extends PullBasedAdapter implements IFeedAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -39,17 +36,21 @@
     private boolean isAlterRequested = false;
     private Map<String, String> alteredParams = new HashMap<String, String>();
     private String id_prefix = "";
-    private ARecordType recordType;
 
     private IPullBasedFeedClient rssFeedClient;
-
-    public static final String KEY_RSS_URL = "url";
-    public static final String KEY_INTERVAL = "interval";
+    private ARecordType recordType;
 
     public boolean isStopRequested() {
         return isStopRequested;
     }
 
+    public RSSFeedAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
+            throws AsterixException {
+        super(configuration, ctx);
+        id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
+        this.recordType = recordType;
+    }
+
     public void setStopRequested(boolean isStopRequested) {
         this.isStopRequested = isStopRequested;
     }
@@ -66,25 +67,6 @@
         isStopRequested = true;
     }
 
-    @Override
-    public AdapterType getAdapterType() {
-        return AdapterType.READ;
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        configuration = arguments;
-        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
-        if (rssURLProperty == null) {
-            throw new IllegalArgumentException("no rss url provided");
-        }
-        initializeFeedURLs(rssURLProperty);
-        configurePartitionConstraints();
-        recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
-                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
-                false);
-    }
-
     private void initializeFeedURLs(String rssURLProperty) {
         feedURLs.clear();
         String[] feedURLProperty = rssURLProperty.split(",");
@@ -94,22 +76,12 @@
     }
 
     protected void reconfigure(Map<String, String> arguments) {
-        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+        String rssURLProperty = configuration.get("KEY_RSS_URL");
         if (rssURLProperty != null) {
             initializeFeedURLs(rssURLProperty);
         }
     }
 
-    protected void configurePartitionConstraints() {
-        partitionConstraint = new AlgebricksCountPartitionConstraint(feedURLs.size());
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        this.ctx = ctx;
-        id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
-    }
-
     public boolean isAlterRequested() {
         return isAlterRequested;
     }
@@ -126,17 +98,8 @@
         return rssFeedClient;
     }
 
-    @Override
-    public ARecordType getAdapterOutputType() {
+    public ARecordType getRecordType() {
         return recordType;
     }
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (partitionConstraint == null) {
-            configurePartitionConstraints();
-        }
-        return partitionConstraint;
-    }
-
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
index d89674f..8a4b301 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
@@ -18,6 +18,7 @@
 import java.net.URL;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 
 import com.sun.syndication.feed.synd.SyndEntryImpl;
@@ -75,16 +76,16 @@
         fetcher.addFetcherEventListener(listener);
         mutableFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableString(null),
                 new AMutableString(null) };
-        recordType = adapter.getAdapterOutputType();
+        recordType = adapter.getRecordType();
         mutableRecord = new AMutableRecord(recordType, mutableFields);
         tupleFieldValues = new String[recordType.getFieldNames().length];
     }
 
     @Override
-    public boolean setNextRecord() throws Exception {
+    public InflowState setNextRecord() throws Exception {
         SyndEntryImpl feedEntry = getNextRSSFeed();
         if (feedEntry == null) {
-            return false;
+            return InflowState.DATA_NOT_AVAILABLE;
         }
         tupleFieldValues[0] = idPrefix + ":" + id;
         tupleFieldValues[1] = feedEntry.getTitle();
@@ -96,7 +97,7 @@
             mutableRecord.setValueAtPos(i, mutableFields[i]);
         }
         id++;
-        return true;
+        return InflowState.DATA_AVAILABLE;
     }
 
     private SyndEntryImpl getNextRSSFeed() throws Exception {
@@ -138,6 +139,18 @@
 
     }
 
+    @Override
+    public boolean alter(Map<String, String> configuration) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public void stop() {
+        // TODO Auto-generated method stub
+
+    }
+
 }
 
 class FetcherEventListenerImpl implements FetcherListener {
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java
new file mode 100644
index 0000000..f89a7ff
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public abstract class StreamBasedAdapter implements IDatasourceAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+
+    public abstract InputStream getInputStream(int partition) throws IOException;
+
+    protected final ITupleParser tupleParser;
+    protected final IAType sourceDatatype;
+    protected IHyracksTaskContext ctx;
+    protected AdapterRuntimeManager runtimeManager;
+
+    public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx)
+            throws HyracksDataException {
+        this.tupleParser = parserFactory.createTupleParser(ctx);
+        this.sourceDatatype = sourceDatatype;
+    }
+
+    @Override
+    public void start(int partition, IFrameWriter writer) throws Exception {
+        InputStream in = getInputStream(partition);
+        tupleParser.parse(in, writer);
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java
deleted file mode 100644
index c36dc03..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java
+++ /dev/null
@@ -1,43 +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.external.feed.lifecycle;
-
-import java.util.Map;
-
-/**
- * A feed control message containing the altered values for
- * adapter configuration parameters. This message is dispatched
- * to all runtime instances of the feed's adapter.
- */
-public class AlterFeedMessage extends FeedMessage {
-
-    private static final long serialVersionUID = 1L;
-
-    private final Map<String, String> alteredConfParams;
-
-    public AlterFeedMessage(Map<String, String> alteredConfParams) {
-        super(MessageType.ALTER);
-        this.alteredConfParams = alteredConfParams;
-    }
-
-    @Override
-    public MessageType getMessageType() {
-        return MessageType.ALTER;
-    }
-
-    public Map<String, String> getAlteredConfParams() {
-        return alteredConfParams;
-    }
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedId.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedId.java
deleted file mode 100644
index 94e679d..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/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.external.feed.lifecycle;
-
-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-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java
deleted file mode 100644
index 5b3ed35..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java
+++ /dev/null
@@ -1,71 +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.external.feed.lifecycle;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-/**
- * Handle (de-)registration of feeds for delivery of control messages.
- */
-public class FeedManager implements IFeedManager {
-
-    public static FeedManager INSTANCE = new FeedManager();
-
-    private FeedManager() {
-
-    }
-
-    private Map<FeedId, Set<LinkedBlockingQueue<IFeedMessage>>> outGoingMsgQueueMap = new HashMap<FeedId, Set<LinkedBlockingQueue<IFeedMessage>>>();
-
-    @Override
-    public void deliverMessage(FeedId feedId, IFeedMessage feedMessage) throws AsterixException {
-        Set<LinkedBlockingQueue<IFeedMessage>> operatorQueues = outGoingMsgQueueMap.get(feedId);
-        try {
-            if (operatorQueues != null) {
-                for (LinkedBlockingQueue<IFeedMessage> queue : operatorQueues) {
-                    queue.put(feedMessage);
-                }
-            } 
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-    }
-
-    @Override
-    public void registerFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue) {
-        Set<LinkedBlockingQueue<IFeedMessage>> feedQueues = outGoingMsgQueueMap.get(feedId);
-        if (feedQueues == null) {
-            feedQueues = new HashSet<LinkedBlockingQueue<IFeedMessage>>();
-        }
-        feedQueues.add(queue);
-        outGoingMsgQueueMap.put(feedId, feedQueues);
-    }
-
-    @Override
-    public void unregisterFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue) {
-        Set<LinkedBlockingQueue<IFeedMessage>> feedQueues = outGoingMsgQueueMap.get(feedId);
-        if (feedQueues == null || !feedQueues.contains(queue)) {
-            throw new IllegalArgumentException(" Unable to de-register feed message queue. Unknown feedId " + feedId);
-        }
-        feedQueues.remove(queue);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java
deleted file mode 100644
index 2febbe4..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java
+++ /dev/null
@@ -1,59 +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.external.feed.lifecycle;
-
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-/**
- * Handle (de-)registration of feeds for delivery of control messages.
- */
-public interface IFeedManager {
-
-    /**
-     * Register an input message queue for a feed specified by feedId.
-     * All messages sent to a feed are directed to the registered queue(s).
-     * 
-     * @param feedId
-     *            an identifier for the feed dataset.
-     * @param queue
-     *            an input message queue for receiving control messages.
-     */
-    public void registerFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue);
-
-    /**
-     * Unregister an input message queue for a feed specified by feedId.
-     * A feed prior to finishing should unregister all previously registered queue(s)
-     * as it is no longer active and thus need not process any control messages.
-     * 
-     * @param feedId
-     *            an identifier for the feed dataset.
-     * @param queue
-     *            an input message queue for receiving control messages.
-     */
-    public void unregisterFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue);
-
-    /**
-     * Deliver a message to a feed with a given feedId.
-     * 
-     * @param feedId
-     *            identifier for the feed dataset.
-     * @param feedMessage
-     *            control message that needs to be delivered.
-     * @throws Exception
-     */
-    public void deliverMessage(FeedId feedId, IFeedMessage feedMessage) throws AsterixException;
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java
new file mode 100755
index 0000000..3fd96ec
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class ExternalFunction implements IExternalFunction {
+
+    protected final IExternalFunctionInfo finfo;
+    protected final IFunctionFactory externalFunctionFactory;
+    protected final IExternalFunction externalFunction;
+    protected final ICopyEvaluatorFactory[] evaluatorFactories;
+    protected final IDataOutputProvider out;
+    protected final ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
+    protected final ICopyEvaluator[] argumentEvaluators;
+    protected final JavaFunctionHelper functionHelper;
+
+    public ExternalFunction(IExternalFunctionInfo finfo, ICopyEvaluatorFactory args[],
+            IDataOutputProvider outputProvider) throws AlgebricksException {
+        this.finfo = finfo;
+        this.evaluatorFactories = args;
+        this.out = outputProvider;
+        argumentEvaluators = new ICopyEvaluator[args.length];
+        for (int i = 0; i < args.length; i++) {
+            argumentEvaluators[i] = args[i].createEvaluator(inputVal);
+        }
+        functionHelper = new JavaFunctionHelper(finfo, outputProvider);
+
+        String[] fnameComponents = finfo.getFunctionIdentifier().getName().split("#");
+        String functionLibary = fnameComponents[0];
+        String dataverse = finfo.getFunctionIdentifier().getNamespace();
+        ClassLoader libraryClassLoader = ExternalLibraryManager.getLibraryClassLoader(dataverse, functionLibary);
+        String classname = finfo.getFunctionBody().trim();
+        Class clazz;
+        try {
+            clazz = Class.forName(classname, true, libraryClassLoader);
+            externalFunctionFactory = (IFunctionFactory) clazz.newInstance();
+            externalFunction = externalFunctionFactory.getExternalFunction();
+        } catch (Exception e) {
+            throw new AlgebricksException(" Unable to load/instantiate class " + classname, e);
+        }
+    }
+
+    public static ISerializerDeserializer getSerDe(Object typeInfo) {
+        return AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(typeInfo);
+    }
+
+    public IExternalFunctionInfo getFinfo() {
+        return finfo;
+    }
+
+    public void setArguments(IFrameTupleReference tuple) throws AlgebricksException, IOException, AsterixException {
+        for (int i = 0; i < evaluatorFactories.length; i++) {
+            inputVal.reset();
+            argumentEvaluators[i].evaluate(tuple);
+            functionHelper.setArgument(i, inputVal.getByteArray());
+        }
+    }
+
+    @Override
+    public void deinitialize() {
+        externalFunction.deinitialize();
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java
new file mode 100755
index 0000000..b5888a6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+
+public class ExternalFunctionDescriptorProvider {
+
+    public static IFunctionDescriptor getExternalFunctionDescriptor(IExternalFunctionInfo finfo) throws AsterixException {
+        switch (finfo.getKind()) {
+            case SCALAR:
+                return new ExternalScalarFunctionDescriptor(finfo);
+            case AGGREGATE:
+            case UNNEST:
+                throw new AsterixException("Unsupported function kind :" + finfo.getKind());
+            default:
+                break;
+        }
+        return null;
+    }
+
+}
+
+class ExternalScalarFunctionDescriptor extends AbstractScalarFunctionDynamicDescriptor implements IFunctionDescriptor {
+
+    private final IFunctionInfo finfo;
+    private ICopyEvaluatorFactory evaluatorFactory;
+    private ICopyEvaluatorFactory[] args;
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        evaluatorFactory = new ExternalScalarFunctionEvaluatorFactory((IExternalFunctionInfo) finfo, args);
+        return evaluatorFactory;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return finfo.getFunctionIdentifier();
+    }
+
+    public ExternalScalarFunctionDescriptor(IFunctionInfo finfo) {
+        this.finfo = finfo;
+    }
+
+}
+
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java
new file mode 100755
index 0000000..db6c224
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java
@@ -0,0 +1,65 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ExternalFunctionProvider {
+
+    private static Map<IExternalFunctionInfo, ExternalScalarFunction> functionRepo = new HashMap<IExternalFunctionInfo, ExternalScalarFunction>();
+
+    public static IExternalFunction getExternalFunctionEvaluator(IExternalFunctionInfo finfo,
+            ICopyEvaluatorFactory args[], IDataOutputProvider outputProvider) throws AlgebricksException {
+        switch (finfo.getKind()) {
+            case SCALAR:
+                ExternalScalarFunction function = functionRepo.get(finfo);
+                function = new ExternalScalarFunction(finfo, args, outputProvider);
+                // functionRepo.put(finfo, function);
+                return function;
+            case AGGREGATE:
+            case UNNEST:
+                throw new IllegalArgumentException(" not supported function kind" + finfo.getKind());
+            default:
+                throw new IllegalArgumentException(" unknown function kind" + finfo.getKind());
+        }
+    }
+}
+
+class ExternalScalarFunction extends ExternalFunction implements IExternalScalarFunction, ICopyEvaluator {
+
+    public ExternalScalarFunction(IExternalFunctionInfo finfo, ICopyEvaluatorFactory args[],
+            IDataOutputProvider outputProvider) throws AlgebricksException {
+        super(finfo, args, outputProvider);
+        try {
+            initialize(functionHelper);
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+        try {
+            setArguments(tuple);
+            evaluate(functionHelper);
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    public void evaluate(IFunctionHelper argumentProvider) throws Exception {
+        ((IExternalScalarFunction) externalFunction).evaluate(argumentProvider);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) throws Exception {
+        ((IExternalScalarFunction) externalFunction).initialize(functionHelper);
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
new file mode 100755
index 0000000..520020d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ExternalLibraryManager {
+
+    private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<String, ClassLoader>();
+
+    public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader) {
+        String key = dataverseName + "." + libraryName;
+        synchronized (libraryClassLoaders) {
+            if (libraryClassLoaders.get(dataverseName) != null) {
+                throw new IllegalStateException("library class loader already registered!");
+            }
+            libraryClassLoaders.put(key, classLoader);
+        }
+    }
+
+    public static void deregisterLibraryClassLoader(String dataverseName, String libraryName) {
+        String key = dataverseName + "." + libraryName;
+        synchronized (libraryClassLoaders) {
+            if (libraryClassLoaders.get(dataverseName) != null) {
+                libraryClassLoaders.remove(key);
+            }
+        }
+    }
+
+    public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
+        String key = dataverseName + "." + libraryName;
+        synchronized (libraryClassLoaders) {
+            return libraryClassLoaders.get(key);
+        }
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
new file mode 100755
index 0000000..a185000
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ExternalScalarFunctionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+    private final IExternalFunctionInfo finfo;
+    private final ICopyEvaluatorFactory[] args;
+
+    public ExternalScalarFunctionEvaluatorFactory(IExternalFunctionInfo finfo, ICopyEvaluatorFactory[] args)
+            throws AlgebricksException {
+        this.finfo = finfo;
+        this.args = args;
+    }
+
+    @Override
+    public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+        return (ExternalScalarFunction) ExternalFunctionProvider.getExternalFunctionEvaluator(finfo, args, output);
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java
new file mode 100755
index 0000000..e667828
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IExternalFunction {
+
+    public void deinitialize();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java
new file mode 100755
index 0000000..09451d8
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IExternalScalarFunction extends IExternalFunction {
+
+    public void initialize(IFunctionHelper functionHelper) throws Exception;
+
+    public void evaluate(IFunctionHelper functionHelper) throws Exception;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java
new file mode 100755
index 0000000..f67957d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IFunctionFactory {
+
+    public IExternalFunction getExternalFunction();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java
new file mode 100755
index 0000000..0192e33
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public interface IFunctionHelper {
+
+    public IJObject getArgument(int index);
+
+    public IJObject getResultObject();
+
+    public void setResult(IJObject result) throws IOException, AsterixException;
+    
+    public IJObject getObject(JTypeTag jtypeTag);
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java
new file mode 100755
index 0000000..fee002e
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.base.AOrderedList;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.IAObject;
+
+public interface IResultCollector {
+
+    public void writeIntResult(int result) throws AsterixException;
+
+    public void writeFloatResult(float result) throws AsterixException;
+
+    public void writeDoubleResult(double result) throws AsterixException;
+
+    public void writeStringResult(String result) throws AsterixException;
+
+    public void writeRecordResult(ARecord result) throws AsterixException;
+
+    public void writeListResult(AOrderedList list) throws AsterixException;
+
+    public IAObject getComplexTypeResultHolder();
+    
+    public DataOutput getDataOutput();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java
new file mode 100644
index 0000000..ad56981
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java
@@ -0,0 +1,115 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInterval;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLong;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRectangle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectFactory;
+
+public class JTypeObjectFactory implements IObjectFactory<IJObject, IAType> {
+
+    @Override
+    public IJObject create(IAType type) {
+        IJObject retValue = null;
+        switch (type.getTypeTag()) {
+            case INT32:
+                retValue = new JInt(0);
+                break;
+            case STRING:
+                retValue = new JString("");
+                break;
+            case FLOAT:
+                retValue = new JFloat(0);
+                break;
+            case DOUBLE:
+                retValue = new JDouble(0);
+                break;
+            case BOOLEAN:
+                retValue = new JBoolean(false);
+                break;
+            case CIRCLE:
+                retValue = new JCircle(new JPoint(0, 0), 0);
+                break;
+            case POINT:
+                retValue = new JPoint(0, 0);
+                break;
+            case POINT3D:
+                retValue = new JPoint3D(0, 0, 0);
+                break;
+            case POLYGON:
+                retValue = new JPolygon(new ArrayList<JPoint>());
+                break;
+            case LINE:
+                retValue = new JLine(new JPoint(0, 0), new JPoint(0, 0));
+                break;
+            case RECTANGLE:
+                retValue = new JRectangle(new JPoint(0, 0), new JPoint(1, 1));
+                break;
+            case DATE:
+                retValue = new JDate(0);
+                break;
+            case DATETIME:
+                retValue = new JDateTime(0);
+                break;
+            case DURATION:
+                retValue = new JDuration(0, 0);
+                break;
+            case INTERVAL:
+                retValue = new JInterval(0, 0);
+                break;
+            case TIME:
+                retValue = new JTime(0);
+                break;
+            case INT64:
+                retValue = new JLong(0);
+                break;
+            case ORDEREDLIST:
+                AOrderedListType ot = (AOrderedListType) type;
+                IAType orderedItemType = ot.getItemType();
+                IJObject orderedItemObject = create(orderedItemType);
+                retValue = new JOrderedList(orderedItemObject);
+                break;
+            case UNORDEREDLIST:
+                AUnorderedListType ut = (AUnorderedListType) type;
+                IAType unorderedItemType = ut.getItemType();
+                IJObject unorderedItemObject = create(unorderedItemType);
+                retValue = new JUnorderedList(unorderedItemObject);
+                break;
+            case RECORD:
+                IAType[] fieldTypes = ((ARecordType) type).getFieldTypes();
+                IJObject[] fieldObjects = new IJObject[fieldTypes.length];
+                int index = 0;
+                for (IAType fieldType : fieldTypes) {
+                    fieldObjects[index] = create(fieldType);
+                    index++;
+                }
+                retValue = new JRecord((ARecordType) type, fieldObjects);
+
+                break;
+        }
+        return retValue;
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java
new file mode 100644
index 0000000..8aed123
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JObjectUtil;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleDataInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+import edu.uci.ics.asterix.om.util.container.ListObjectPool;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class JavaFunctionHelper implements IFunctionHelper {
+
+    private final IExternalFunctionInfo finfo;
+    private final IDataOutputProvider outputProvider;
+    private IJObject[] arguments;
+    private IJObject resultHolder;
+    private IAObject innerResult;
+    private ISerializerDeserializer resultSerde;
+    private IObjectPool<IJObject, IAType> objectPool = new ListObjectPool<IJObject, IAType>(new JTypeObjectFactory());
+    byte[] buffer = new byte[32768];
+    ByteArrayAccessibleInputStream bis = new ByteArrayAccessibleInputStream(buffer, 0, buffer.length);
+    ByteArrayAccessibleDataInputStream dis = new ByteArrayAccessibleDataInputStream(bis);
+
+    public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider)
+            throws AlgebricksException {
+        this.finfo = finfo;
+        this.outputProvider = outputProvider;
+        List<IAType> params = finfo.getParamList();
+        arguments = new IJObject[params.size()];
+        int index = 0;
+        for (IAType param : params) {
+            this.arguments[index] = objectPool.allocate(param);
+            index++;
+        }
+        resultHolder = objectPool.allocate(finfo.getReturnType());
+    }
+
+    @Override
+    public IJObject getArgument(int index) {
+        return arguments[index];
+    }
+
+    @Override
+    public void setResult(IJObject result) throws IOException, AsterixException {
+        IAObject obj = result.getIAObject();
+        try {
+            outputProvider.getDataOutput().writeByte(obj.getType().getTypeTag().serialize());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+
+        if (obj.getType().getTypeTag().equals(ATypeTag.RECORD)) {
+            ARecordType recType = (ARecordType) obj.getType();
+            if (recType.isOpen()) {
+                writeOpenRecord((JRecord) result, outputProvider.getDataOutput());
+            } else {
+                resultSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(recType);
+                resultSerde.serialize(obj, outputProvider.getDataOutput());
+            }
+        } else {
+            resultSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(obj.getType());
+            resultSerde.serialize(obj, outputProvider.getDataOutput());
+        }
+        reset();
+    }
+
+    private void writeOpenRecord(JRecord jRecord, DataOutput dataOutput) throws AsterixException, IOException {
+        ARecord aRecord = (ARecord) jRecord.getIAObject();
+        RecordBuilder recordBuilder = new RecordBuilder();
+        ARecordType recordType = aRecord.getType();
+        recordBuilder.reset(recordType);
+        ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        List<Boolean> openField = jRecord.getOpenField();
+
+        int fieldIndex = 0;
+        int closedFieldId = 0;
+        for (IJObject field : jRecord.getFields()) {
+            fieldValue.reset();
+            switch (field.getTypeTag()) {
+                case RECORD:
+                    ARecordType recType = (ARecordType) field.getIAObject().getType();
+                    if (recType.isOpen()) {
+                        fieldValue.getDataOutput().writeByte(recType.getTypeTag().serialize());
+                        writeOpenRecord((JRecord) field, fieldValue.getDataOutput());
+                    } else {
+                        AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
+                                field.getIAObject().getType()).serialize(field.getIAObject(),
+                                fieldValue.getDataOutput());
+                    }
+                    break;
+                default:
+                    AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(field.getIAObject().getType())
+                            .serialize(field.getIAObject(), fieldValue.getDataOutput());
+                    break;
+            }
+            if (openField.get(fieldIndex)) {
+                String fName = jRecord.getFieldNames().get(fieldIndex);
+                fieldName.reset();
+                AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING).serialize(
+                        new AString(fName), fieldName.getDataOutput());
+                recordBuilder.addField(fieldName, fieldValue);
+            } else {
+                recordBuilder.addField(closedFieldId, fieldValue);
+                closedFieldId++;
+            }
+            fieldIndex++;
+        }
+
+        recordBuilder.write(dataOutput, false);
+
+    }
+
+    private void reset() {
+        for (IJObject jObject : arguments) {
+            switch (jObject.getTypeTag()) {
+                case RECORD:
+                    reset((JRecord) jObject);
+                    break;
+            }
+        }
+        switch (resultHolder.getTypeTag()) {
+            case RECORD:
+                reset((JRecord) resultHolder);
+                break;
+        }
+    }
+
+    private void reset(JRecord jRecord) {
+        List<IJObject> fields = ((JRecord) jRecord).getFields();
+        for (IJObject field : fields) {
+            switch (field.getTypeTag()) {
+                case RECORD:
+                    reset((JRecord) field);
+                    break;
+            }
+        }
+        jRecord.close();
+    }
+
+    public void setArgument(int index, byte[] argument) throws IOException, AsterixException {
+        bis.setContent(argument, 1, argument.length);
+        IAType type = finfo.getParamList().get(index);
+        arguments[index] = JObjectUtil.getJType(type.getTypeTag(), type, dis, objectPool);
+    }
+
+    @Override
+    public IJObject getResultObject() {
+        return resultHolder;
+    }
+
+    @Override
+    public IJObject getObject(JTypeTag jtypeTag) {
+        IJObject retValue = null;
+        switch (jtypeTag) {
+            case INT:
+                retValue = objectPool.allocate(BuiltinType.AINT32);
+                break;
+            case STRING:
+                retValue = objectPool.allocate(BuiltinType.ASTRING);
+                break;
+        }
+        return retValue;
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java
new file mode 100755
index 0000000..d53b044
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java
@@ -0,0 +1,147 @@
+/*
+ * Copyright 2009-2012 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.external.library;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableOrderedList;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AOrderedList;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ResultCollector implements IResultCollector {
+
+    private IAObject reusableResultObjectHolder;
+    private ByteBuffer reusableResultBinaryHolder;
+    private IDataOutputProvider outputProvider;
+    private IExternalFunctionInfo finfo;
+
+    public ResultCollector(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider) {
+        this.finfo = finfo;
+        IAType returnType = finfo.getReturnType();
+        reusableResultObjectHolder = allocateResultObjectHolder(returnType);
+        reusableResultBinaryHolder = allocateResultBinaryHolder(returnType);
+        this.outputProvider = outputProvider;
+    }
+
+    private IAObject allocateResultObjectHolder(IAType type) {
+        switch (type.getTypeTag()) {
+            case INT32:
+                return new AMutableInt32(0);
+            case FLOAT:
+                return new AMutableFloat(0f);
+            case DOUBLE:
+                return new AMutableDouble(0);
+            case STRING:
+                return new AMutableString("");
+            case ORDEREDLIST:
+                return new AMutableOrderedList((AOrderedListType) type);
+            case RECORD:
+                IAType[] fieldType = ((ARecordType) type).getFieldTypes();
+                IAObject[] fieldObjects = new IAObject[fieldType.length];
+                for (int i = 0; i < fieldType.length; i++) {
+                    fieldObjects[i] = allocateResultObjectHolder(fieldType[i]);
+                }
+                return new AMutableRecord((ARecordType) type, fieldObjects);
+        }
+        return null;
+    }
+
+    private ByteBuffer allocateResultBinaryHolder(IAType type) {
+        switch (type.getTypeTag()) {
+            case INT32:
+                return ByteBuffer.allocate(4);
+            case FLOAT:
+                return ByteBuffer.allocate(4);
+            case DOUBLE:
+                return ByteBuffer.allocate(8);
+            case STRING:
+                return ByteBuffer.allocate(32 * 1024);
+            case ORDEREDLIST:
+                return ByteBuffer.allocate(32 * 1024);
+            case RECORD:
+                return ByteBuffer.allocate(32 * 1024);
+        }
+        return null;
+    }
+
+    @Override
+    public void writeDoubleResult(double result) throws AsterixException {
+        ((AMutableDouble) reusableResultObjectHolder).setValue(result);
+        serializeResult(reusableResultObjectHolder);
+    }
+
+    @Override
+    public void writeFloatResult(float result) throws AsterixException {
+        ((AMutableDouble) reusableResultObjectHolder).setValue(result);
+        serializeResult(reusableResultObjectHolder);
+    }
+
+    @Override
+    public void writeIntResult(int result) throws AsterixException {
+        ((AMutableInt32) reusableResultObjectHolder).setValue(result);
+        serializeResult(reusableResultObjectHolder);
+    }
+
+    @Override
+    public void writeStringResult(String result) throws AsterixException {
+        ((AMutableString) reusableResultObjectHolder).setValue(result);
+        serializeResult(reusableResultObjectHolder);
+
+    }
+
+    @Override
+    public void writeRecordResult(ARecord result) throws AsterixException {
+        serializeResult(result);
+    }
+
+    @Override
+    public void writeListResult(AOrderedList list) throws AsterixException {
+        serializeResult(list);
+    }
+
+    public IAObject getComplexTypeResultHolder() {
+        return reusableResultObjectHolder;
+    }
+
+    private void serializeResult(IAObject object) throws AsterixException {
+        try {
+            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(finfo.getReturnType()).serialize(
+                    reusableResultObjectHolder, outputProvider.getDataOutput());
+        } catch (HyracksDataException hde) {
+            throw new AsterixException(hde);
+        }
+    }
+
+    @Override
+    public DataOutput getDataOutput() {
+        return outputProvider.getDataOutput();
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java
new file mode 100755
index 0000000..0c6713d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java
@@ -0,0 +1,100 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+
+public class RuntimeExternalFunctionUtil {
+
+    private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<String, ClassLoader>();
+
+    public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader) {
+        String key = dataverseName + "." + libraryName;
+        synchronized (libraryClassLoaders) {
+            if (libraryClassLoaders.get(dataverseName) != null) {
+                throw new IllegalStateException("library class loader already registered!");
+            }
+            libraryClassLoaders.put(key, classLoader);
+        }
+    }
+
+    public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
+        String key = dataverseName + "." + libraryName;
+        synchronized (libraryClassLoaders) {
+            return libraryClassLoaders.get(key);
+        }
+    }
+
+    public static IFunctionDescriptor getFunctionDescriptor(IFunctionInfo finfo) {
+        switch (((IExternalFunctionInfo) finfo).getKind()) {
+            case SCALAR:
+                return getScalarFunctionDescriptor(finfo);
+            case AGGREGATE:
+            case UNNEST:
+            case STATEFUL:
+                throw new NotImplementedException("External " + finfo.getFunctionIdentifier().getName()
+                        + " not supported");
+        }
+        return null;
+    }
+
+    private static AbstractScalarFunctionDynamicDescriptor getScalarFunctionDescriptor(IFunctionInfo finfo) {
+        return new ExternalScalarFunctionDescriptor(finfo);
+    }
+
+    public static ByteBuffer allocateArgumentBuffers(IAType type) {
+        switch (type.getTypeTag()) {
+            case INT32:
+                return ByteBuffer.allocate(4);
+            case STRING:
+                return ByteBuffer.allocate(32 * 1024);
+            default:
+                return ByteBuffer.allocate(32 * 1024);
+        }
+    }
+
+    public static IAObject allocateArgumentObjects(IAType type) {
+        switch (type.getTypeTag()) {
+            case INT32:
+                return new AMutableInt32(0);
+            case STRING:
+                return new AMutableString("");
+            default:
+                return null;
+                /*
+                ARecordType recordType = (ARecordType) type;
+                IAType[] fieldTypes = recordType.getFieldTypes();
+                IAObject[] fields = new IAObject[fieldTypes.length];
+                for (int i = 0; i < fields.length; i++) {
+                    fields[i] = allocateArgumentObjects(fieldTypes[i]);
+                }
+                return new AMutableRecord((ARecordType) type, fields);
+                */
+        }
+    }
+
+    public static File getExternalLibraryDeployDir(String nodeId) {
+        String filePath = null;
+        if (nodeId != null) {
+            filePath = "edu.uci.ics.hyracks.control.nc.NodeControllerService" + "/" + nodeId + "/"
+                    + "applications/asterix/expanded/external-lib/libraries";
+        } else {
+            filePath = "ClusterControllerService" + "/" + "applications/asterix/expanded/external-lib/libraries";
+
+        }
+        return new File(filePath);
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java
new file mode 100644
index 0000000..1561c42
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+
+public interface IJObject {
+
+    public ATypeTag getTypeTag();
+
+    public IAObject getIAObject();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java
new file mode 100644
index 0000000..fcc35eb
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+
+public interface IJType {
+
+      public ATypeTag getTypeTag();
+      
+      public IAObject getIAObject();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java
new file mode 100644
index 0000000..52071af
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java
@@ -0,0 +1,399 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleDataInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInterval;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRectangle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+
+public class JObjectUtil {
+
+    public static IJObject getJType(ATypeTag typeTag, IAType type, ByteArrayAccessibleDataInputStream dis,
+            IObjectPool<IJObject, IAType> objectPool) throws IOException, AsterixException {
+        IJObject jObject;
+
+        switch (typeTag) {
+
+            case INT32: {
+                int v = dis.readInt();
+                jObject = objectPool.allocate(BuiltinType.AINT32);
+                ((JInt) jObject).setValue(v);
+                break;
+            }
+
+            case FLOAT: {
+                float v = dis.readFloat();
+                jObject = objectPool.allocate(BuiltinType.AFLOAT);
+                ((JFloat) jObject).setValue(v);
+                break;
+            }
+
+            case DOUBLE: {
+                double value = dis.readDouble();
+                jObject = objectPool.allocate(BuiltinType.ADOUBLE);
+                ((JDouble) jObject).setValue(value);
+                break;
+            }
+
+            case STRING: {
+                String v = dis.readUTF();
+                jObject = objectPool.allocate(BuiltinType.ASTRING);
+                ((JString) jObject).setValue(v);
+                break;
+            }
+
+            case BOOLEAN:
+                jObject = objectPool.allocate(BuiltinType.ABOOLEAN);
+                ((JBoolean) jObject).setValue(dis.readBoolean());
+                break;
+
+            case DATE: {
+                int d = dis.readInt();
+                jObject = objectPool.allocate(BuiltinType.ADATE);
+                ((JDate) jObject).setValue(d);
+                break;
+            }
+
+            case DATETIME: {
+                jObject = objectPool.allocate(BuiltinType.ADATETIME);
+                long value = dis.readLong();
+                ((JDateTime) jObject).setValue(value);
+                break;
+            }
+
+            case DURATION: {
+                jObject = objectPool.allocate(BuiltinType.ADURATION);
+                int months = dis.readInt();
+                long msecs = dis.readLong();
+                ((JDuration) jObject).setValue(months, msecs);
+                break;
+            }
+
+            case TIME: {
+                jObject = objectPool.allocate(BuiltinType.ATIME);
+                int time = dis.readInt();
+                ((JTime) jObject).setValue(time);
+                break;
+            }
+
+            case INTERVAL: {
+                jObject = objectPool.allocate(BuiltinType.AINTERVAL);
+                long start = dis.readLong();
+                long end = dis.readLong();
+                byte intervalType = dis.readByte();
+                ((JInterval) jObject).setValue(start, end, intervalType);
+                break;
+            }
+
+            case CIRCLE: {
+                jObject = objectPool.allocate(BuiltinType.ACIRCLE);
+                double x = dis.readDouble();
+                double y = dis.readDouble();
+                double radius = dis.readDouble();
+                JPoint jpoint = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                jpoint.setValue(x, y);
+                ((JCircle) jObject).setValue(jpoint, radius);
+                break;
+            }
+
+            case POINT: {
+                jObject = objectPool.allocate(BuiltinType.APOINT);
+                double x = dis.readDouble();
+                double y = dis.readDouble();
+                ((JPoint) jObject).setValue(x, y);
+                break;
+            }
+
+            case POINT3D: {
+                jObject = objectPool.allocate(BuiltinType.APOINT3D);
+                double x = dis.readDouble();
+                double y = dis.readDouble();
+                double z = dis.readDouble();
+                ((JPoint3D) jObject).setValue(x, y, z);
+                break;
+            }
+
+            case LINE: {
+                jObject = objectPool.allocate(BuiltinType.ALINE);
+                double x1 = dis.readDouble();
+                double y1 = dis.readDouble();
+                double x2 = dis.readDouble();
+                double y2 = dis.readDouble();
+                JPoint jpoint1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                jpoint1.setValue(x1, y1);
+                JPoint jpoint2 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                jpoint2.setValue(x2, y2);
+                ((JLine) jObject).setValue(jpoint1, jpoint2);
+                break;
+            }
+
+            case POLYGON: {
+                jObject = objectPool.allocate(BuiltinType.APOLYGON);
+                short numberOfPoints = dis.readShort();
+                List<JPoint> points = new ArrayList<JPoint>();
+                for (int i = 0; i < numberOfPoints; i++) {
+                    JPoint p1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                    p1.setValue(dis.readDouble(), dis.readDouble());
+                    points.add(p1);
+                }
+                ((JPolygon) jObject).setValue(points);
+                break;
+            }
+
+            case RECTANGLE: {
+                jObject = objectPool.allocate(BuiltinType.ARECTANGLE);
+                double x1 = dis.readDouble();
+                double y1 = dis.readDouble();
+                double x2 = dis.readDouble();
+                double y2 = dis.readDouble();
+                JPoint jpoint1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                jpoint1.setValue(x1, y1);
+                JPoint jpoint2 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+                jpoint2.setValue(x2, y2);
+                ((JRectangle) jObject).setValue(jpoint1, jpoint2);
+                break;
+            }
+
+            case UNORDEREDLIST: {
+                AUnorderedListType listType = (AUnorderedListType) type;
+                IAType elementType = listType.getItemType();
+                jObject = objectPool.allocate(listType);
+
+                boolean fixedSize = false;
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(dis.readByte());
+                switch (tag) {
+                    case STRING:
+                    case RECORD:
+                    case ORDEREDLIST:
+                    case UNORDEREDLIST:
+                    case ANY:
+                        fixedSize = false;
+                        break;
+                    default:
+                        fixedSize = true;
+                        break;
+                }
+                dis.readInt(); // list size
+                int numberOfitems;
+                numberOfitems = dis.readInt();
+                if (numberOfitems > 0) {
+                    if (!fixedSize) {
+                        for (int i = 0; i < numberOfitems; i++)
+                            dis.readInt();
+                    }
+                    for (int i = 0; i < numberOfitems; i++) {
+                        IJObject v = (IJObject) getJType(elementType.getTypeTag(), elementType, dis, objectPool);
+                        ((JUnorderedList) jObject).add(v);
+                    }
+                }
+
+                break;
+            }
+            case ORDEREDLIST: {
+                AOrderedListType listType = (AOrderedListType) type;
+                IAType elementType = listType.getItemType();
+                jObject = objectPool.allocate(listType);
+                boolean fixedSize = false;
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(dis.readByte());
+                switch (tag) {
+                    case STRING:
+                    case RECORD:
+                    case ORDEREDLIST:
+                    case UNORDEREDLIST:
+                    case ANY:
+                        fixedSize = false;
+                        break;
+                    default:
+                        fixedSize = true;
+                        break;
+                }
+
+                dis.readInt(); // list size
+                int numberOfitems;
+                numberOfitems = dis.readInt();
+                if (numberOfitems > 0) {
+                    if (!fixedSize) {
+                        for (int i = 0; i < numberOfitems; i++)
+                            dis.readInt();
+                    }
+                    for (int i = 0; i < numberOfitems; i++) {
+                        IJObject v = (IJObject) getJType(elementType.getTypeTag(), elementType, dis, objectPool);
+                        ((JOrderedList) jObject).add(v);
+                    }
+                }
+
+                break;
+            }
+            case RECORD:
+                ARecordType recordType = (ARecordType) type;
+                int numberOfSchemaFields = recordType.getFieldTypes().length;
+                byte[] recordBits = dis.getInputStream().getArray();
+                boolean isExpanded = false;
+                int s = dis.getInputStream().getPosition();
+                int recordOffset = s;
+                int openPartOffset = 0;
+                int offsetArrayOffset = 0;
+                int[] fieldOffsets = new int[numberOfSchemaFields];
+                IJObject[] closedFields = new IJObject[numberOfSchemaFields];
+
+                if (recordType == null) {
+                    openPartOffset = s + AInt32SerializerDeserializer.getInt(recordBits, s + 6);
+                    s += 8;
+                    isExpanded = true;
+                } else {
+                    dis.skip(4); // reading length is not required.
+                    if (recordType.isOpen()) {
+                        isExpanded = dis.readBoolean();
+                        if (isExpanded) {
+                            openPartOffset = s + dis.readInt(); // AInt32SerializerDeserializer.getInt(recordBits, s + 6);
+                        } else {
+                            // do nothing s += 6;
+                        }
+                    } else {
+                        // do nothing s += 5;
+                    }
+                }
+
+                if (numberOfSchemaFields > 0) {
+                    int numOfSchemaFields = dis.readInt(); //s += 4;
+                    int nullBitMapOffset = 0;
+                    boolean hasNullableFields = NonTaggedFormatUtil.hasNullableField(recordType);
+                    if (hasNullableFields) {
+                        nullBitMapOffset = dis.getInputStream().getPosition();//s
+                        offsetArrayOffset = dis.getInputStream().getPosition() //s
+                                + (numberOfSchemaFields % 8 == 0 ? numberOfSchemaFields / 8
+                                        : numberOfSchemaFields / 8 + 1);
+                    } else {
+                        offsetArrayOffset = dis.getInputStream().getPosition();
+                    }
+                    for (int i = 0; i < numberOfSchemaFields; i++) {
+                        fieldOffsets[i] = dis.readInt(); // AInt32SerializerDeserializer.getInt(recordBits, offsetArrayOffset) + recordOffset;
+                        // offsetArrayOffset += 4;
+                    }
+                    for (int fieldNumber = 0; fieldNumber < numberOfSchemaFields; fieldNumber++) {
+                        if (hasNullableFields) {
+                            byte b1 = recordBits[nullBitMapOffset + fieldNumber / 8];
+                            int p = 1 << (7 - (fieldNumber % 8));
+                            if ((b1 & p) == 0) {
+                                // set null value (including type tag inside)
+                                //fieldValues.add(nullReference);
+                                continue;
+                            }
+                        }
+                        IAType[] fieldTypes = recordType.getFieldTypes();
+                        ATypeTag fieldValueTypeTag = null;
+
+                        IAType fieldType = fieldTypes[fieldNumber];
+                        if (fieldTypes[fieldNumber].getTypeTag() == ATypeTag.UNION) {
+                            if (NonTaggedFormatUtil.isOptionalField((AUnionType) fieldTypes[fieldNumber])) {
+                                fieldType = ((AUnionType) fieldTypes[fieldNumber]).getUnionList().get(
+                                        NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+                                fieldValueTypeTag = fieldType.getTypeTag();
+                                //                      fieldValueLength = NonTaggedFormatUtil.getFieldValueLength(recordBits,
+                                //                              fieldOffsets[fieldNumber], typeTag, false);
+                            }
+                        } else {
+                            fieldValueTypeTag = fieldTypes[fieldNumber].getTypeTag();
+                        }
+                        closedFields[fieldNumber] = getJType(fieldValueTypeTag, fieldType, dis, objectPool);
+                    }
+                }
+                if (isExpanded) {
+                    int numberOfOpenFields = dis.readInt();
+                    String[] fieldNames = new String[numberOfOpenFields];
+                    IAType[] fieldTypes = new IAType[numberOfOpenFields];
+                    IJObject[] openFields = new IJObject[numberOfOpenFields];
+                    for (int i = 0; i < numberOfOpenFields; i++) {
+                        dis.readInt();
+                        dis.readInt();
+                    }
+                    for (int i = 0; i < numberOfOpenFields; i++) {
+                        fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(dis).getStringValue();
+                        ATypeTag openFieldTypeTag = SerializerDeserializerUtil.deserializeTag(dis);
+                        openFields[i] = getJType(openFieldTypeTag, null, dis, objectPool);
+                        fieldTypes[i] = openFields[i].getIAObject().getType();
+                    }
+                    ARecordType openPartRecType = new ARecordType(null, fieldNames, fieldTypes, true);
+                    if (numberOfSchemaFields > 0) {
+                        ARecordType mergedRecordType = mergeRecordTypes(recordType, openPartRecType);
+                        IJObject[] mergedFields = mergeFields(closedFields, openFields);
+                        jObject = objectPool.allocate(recordType);
+                        return new JRecord(mergedRecordType, mergedFields);
+                    } else {
+                        return new JRecord(recordType, openFields);
+                    }
+                } else {
+                    return new JRecord(recordType, closedFields);
+                }
+
+            default:
+                throw new IllegalStateException("Argument type: " + typeTag);
+        }
+        return jObject;
+    }
+
+    private static IJObject[] mergeFields(IJObject[] closedFields, IJObject[] openFields) {
+        IJObject[] fields = new IJObject[closedFields.length + openFields.length];
+        int i = 0;
+        for (; i < closedFields.length; i++) {
+            fields[i] = closedFields[i];
+        }
+        for (int j = 0; j < openFields.length; j++) {
+            fields[closedFields.length + j] = openFields[j];
+        }
+        return fields;
+    }
+
+    private static ARecordType mergeRecordTypes(ARecordType recType1, ARecordType recType2) throws AsterixException {
+
+        String[] fieldNames = new String[recType1.getFieldNames().length + recType2.getFieldNames().length];
+        IAType[] fieldTypes = new IAType[recType1.getFieldTypes().length + recType2.getFieldTypes().length];
+
+        int i = 0;
+        for (; i < recType1.getFieldNames().length; i++) {
+            fieldNames[i] = recType1.getFieldNames()[i];
+            fieldTypes[i] = recType1.getFieldTypes()[i];
+        }
+
+        for (int j = 0; j < recType2.getFieldNames().length; i++, j++) {
+            fieldNames[i] = recType2.getFieldNames()[j];
+            fieldTypes[i] = recType2.getFieldTypes()[j];
+        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java
new file mode 100644
index 0000000..2fae902
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java
@@ -0,0 +1,875 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AMutableCircle;
+import edu.uci.ics.asterix.om.base.AMutableDate;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableDuration;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInterval;
+import edu.uci.ics.asterix.om.base.AMutableLine;
+import edu.uci.ics.asterix.om.base.AMutableOrderedList;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutablePoint3D;
+import edu.uci.ics.asterix.om.base.AMutablePolygon;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableRectangle;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableTime;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.APoint;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+
+public class JObjects {
+
+    public static abstract class JObject implements IJObject {
+
+        protected IAObject value;
+        protected byte[] bytes;
+
+        protected JObject(IAObject value) {
+            this.value = value;
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return value.getType().getTypeTag();
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return value;
+        }
+    }
+
+    public static final class JInt implements IJObject {
+
+        private AMutableInt32 value;
+
+        public JInt(int value) {
+            this.value = new AMutableInt32(value);
+        }
+
+        public void setValue(int v) {
+            if (value == null) {
+                value = new AMutableInt32(v);
+            } else {
+                ((AMutableInt32) value).setValue(v);
+            }
+        }
+
+        public void setValue(AMutableInt32 v) {
+            value = v;
+        }
+
+        public int getValue() {
+            return ((AMutableInt32) value).getIntegerValue().intValue();
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return BuiltinType.AINT32.getTypeTag();
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return value;
+        }
+
+    }
+
+    public static final class JBoolean implements IJObject {
+
+        private boolean value;
+
+        public JBoolean(boolean value) {
+            this.value = value;
+        }
+
+        public void setValue(boolean value) {
+            this.value = value;
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.BOOLEAN;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return value ? ABoolean.TRUE : ABoolean.FALSE;
+        }
+
+    }
+
+    public static final class JLong extends JObject {
+
+        public JLong(long v) {
+            super(new AMutableInt64(v));
+        }
+
+        public void setValue(long v) {
+            ((AMutableInt64) value).setValue(v);
+        }
+
+        public long getValue() {
+            return ((AMutableInt64) value).getLongValue();
+        }
+
+    }
+
+    public static final class JDouble extends JObject {
+
+        public JDouble(double v) {
+            super(new AMutableDouble(v));
+        }
+
+        public void setValue(double v) {
+            ((AMutableDouble) value).setValue(v);
+        }
+
+        public double getValue() {
+            return ((AMutableDouble) value).getDoubleValue();
+        }
+
+    }
+
+    public static final class JString extends JObject {
+
+        public JString(String v) {
+            super(new AMutableString(v));
+        }
+
+        public void setValue(String v) {
+            ((AMutableString) value).setValue(v);
+        }
+
+        public String getValue() {
+            return ((AMutableString) value).getStringValue();
+        }
+
+    }
+
+    public static final class JFloat implements IJObject {
+
+        private AMutableFloat value;
+
+        public JFloat(float v) {
+            value = new AMutableFloat(v);
+        }
+
+        public void setValue(float v) {
+            ((AMutableFloat) value).setValue(v);
+        }
+
+        public float getValue() {
+            return ((AMutableFloat) value).getFloatValue();
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return BuiltinType.AFLOAT.getTypeTag();
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return value;
+        }
+
+    }
+
+    public static final class JPoint extends JObject {
+
+        public JPoint(double x, double y) {
+            super(new AMutablePoint(x, y));
+        }
+
+        public void setValue(double x, double y) {
+            ((AMutablePoint) value).setValue(x, y);
+        }
+
+        public double getXValue() {
+            return ((AMutablePoint) value).getX();
+        }
+
+        public double getYValue() {
+            return ((AMutablePoint) value).getY();
+        }
+
+        public IAObject getValue() {
+            return value;
+        }
+
+        @Override
+        public String toString() {
+            return value.toString();
+        }
+    }
+
+    public static final class JRectangle implements IJObject {
+
+        private AMutableRectangle rect;
+
+        public JRectangle(JPoint p1, JPoint p2) {
+            rect = new AMutableRectangle((APoint) p1.getValue(), (APoint) p2.getValue());
+        }
+
+        public void setValue(JPoint p1, JPoint p2) {
+            this.rect.setValue((APoint) p1.getValue(), (APoint) p2.getValue());
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.RECTANGLE;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return rect;
+        }
+
+        @Override
+        public String toString() {
+            return rect.toString();
+        }
+
+    }
+
+    public static final class JTime implements IJObject {
+
+        private AMutableTime time;
+
+        public JTime(int timeInMillsec) {
+            time = new AMutableTime(timeInMillsec);
+        }
+
+        public void setValue(int timeInMillsec) {
+            time.setValue(timeInMillsec);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.TIME;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return time;
+        }
+
+        @Override
+        public String toString() {
+            return time.toString();
+        }
+
+    }
+
+    public static final class JInterval implements IJObject {
+
+        private AMutableInterval interval;
+
+        public JInterval(long intervalStart, long intervalEnd) {
+            interval = new AMutableInterval(intervalStart, intervalEnd, (byte) 0);
+        }
+
+        public void setValue(long intervalStart, long intervalEnd, byte typetag) {
+            interval.setValue(intervalStart, intervalEnd, typetag);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.INTERVAL;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return interval;
+        }
+
+        @Override
+        public String toString() {
+            return interval.toString();
+        }
+
+        public long getIntervalStart() {
+            return interval.getIntervalStart();
+        }
+
+        public long getIntervalEnd() {
+            return interval.getIntervalEnd();
+        }
+
+        public short getIntervalType() {
+            return interval.getIntervalType();
+        }
+
+    }
+
+    public static final class JDate implements IJObject {
+
+        private AMutableDate date;
+
+        public JDate(int chrononTimeInDays) {
+            date = new AMutableDate(chrononTimeInDays);
+        }
+
+        public void setValue(int chrononTimeInDays) {
+            date.setValue(chrononTimeInDays);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.DATE;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return date;
+        }
+
+        @Override
+        public String toString() {
+            return date.toString();
+        }
+
+    }
+
+    public static final class JDateTime implements IJObject {
+
+        private AMutableDateTime dateTime;
+
+        public JDateTime(long chrononTime) {
+            dateTime = new AMutableDateTime(chrononTime);
+        }
+
+        public void setValue(long chrononTime) {
+            dateTime.setValue(chrononTime);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.DATETIME;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return dateTime;
+        }
+
+        @Override
+        public String toString() {
+            return dateTime.toString();
+        }
+
+    }
+
+    public static final class JDuration implements IJObject {
+
+        private AMutableDuration duration;
+
+        public JDuration(int months, long milliseconds) {
+            duration = new AMutableDuration(months, milliseconds);
+        }
+
+        public void setValue(int months, long milliseconds) {
+            duration.setValue(months, milliseconds);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.DURATION;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return duration;
+        }
+
+        @Override
+        public String toString() {
+            return duration.toString();
+        }
+
+    }
+
+    public static final class JPolygon implements IJObject {
+
+        private AMutablePolygon polygon;
+        private List<JPoint> points;
+
+        public JPolygon(List<JPoint> points) {
+            this.points = points;
+        }
+
+        public void setValue(List<JPoint> points) {
+            this.points = points;
+            polygon = null;
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.POLYGON;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            if (polygon == null) {
+                APoint[] pts = new APoint[points.size()];
+                int index = 0;
+                for (JPoint p : points) {
+                    pts[index++] = (APoint) p.getIAObject();
+                }
+                polygon = new AMutablePolygon(pts);
+            }
+            return polygon;
+        }
+
+        @Override
+        public String toString() {
+            return getIAObject().toString();
+        }
+
+    }
+
+    public static final class JCircle implements IJObject {
+
+        private AMutableCircle circle;
+
+        public JCircle(JPoint center, double radius) {
+            circle = new AMutableCircle((APoint) center.getIAObject(), radius);
+        }
+
+        public void setValue(JPoint center, double radius) {
+            circle.setValue((APoint) center.getIAObject(), radius);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.CIRCLE;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return circle;
+        }
+
+        @Override
+        public String toString() {
+            return circle.toString();
+        }
+
+    }
+
+    public static final class JLine implements IJObject {
+
+        private AMutableLine line;
+
+        public JLine(JPoint p1, JPoint p2) {
+            line = new AMutableLine((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+        }
+
+        public void setValue(JPoint p1, JPoint p2) {
+            line.setValue((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.LINE;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return line;
+        }
+
+        @Override
+        public String toString() {
+            return line.toString();
+        }
+
+    }
+
+    public static final class JPoint3D implements IJObject {
+
+        private AMutablePoint3D value;
+
+        public JPoint3D(double x, double y, double z) {
+            value = new AMutablePoint3D(x, y, z);
+        }
+
+        public void setValue(double x, double y, double z) {
+            value.setValue(x, y, z);
+        }
+
+        public double getXValue() {
+            return ((AMutablePoint3D) value).getX();
+        }
+
+        public double getYValue() {
+            return ((AMutablePoint3D) value).getY();
+        }
+
+        public double getZValue() {
+            return ((AMutablePoint3D) value).getZ();
+        }
+
+        public IAObject getValue() {
+            return value;
+        }
+
+        @Override
+        public String toString() {
+            return value.toString();
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.POINT3D;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            return value;
+        }
+    }
+
+    public static final class JOrderedList implements IJObject {
+
+        private AOrderedListType listType;
+        private List<IJObject> jObjects;
+
+        public JOrderedList(IJObject jObject) {
+            this.listType = new AOrderedListType(jObject.getIAObject().getType(), null);
+            this.jObjects = new ArrayList<IJObject>();
+        }
+
+        public void add(IJObject jObject) {
+            jObjects.add(jObject);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.ORDEREDLIST;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            AMutableOrderedList v = new AMutableOrderedList(listType);
+            for (IJObject jObj : jObjects) {
+                v.add(jObj.getIAObject());
+            }
+            return v;
+        }
+
+        public AOrderedListType getListType() {
+            return listType;
+        }
+
+        public void addAll(Collection<IJObject> jObjectCollection) {
+            jObjects.addAll(jObjectCollection);
+        }
+
+        public void clear() {
+            jObjects.clear();
+        }
+
+        public IJObject getElement(int index) {
+            return jObjects.get(index);
+        }
+
+        public int size() {
+            return jObjects.size();
+        }
+
+    }
+
+    public static final class JUnorderedList implements IJObject {
+
+        private AUnorderedListType listType;
+        private List<IJObject> jObjects;
+
+        public JUnorderedList(IJObject jObject) {
+            this.listType = new AUnorderedListType(jObject.getIAObject().getType(), null);
+            this.jObjects = new ArrayList<IJObject>();
+        }
+
+        public void add(IJObject jObject) {
+            jObjects.add(jObject);
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.UNORDEREDLIST;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            AMutableUnorderedList v = new AMutableUnorderedList(listType);
+            for (IJObject jObj : jObjects) {
+                v.add(jObj.getIAObject());
+            }
+            return v;
+        }
+
+        public AUnorderedListType getListType() {
+            return listType;
+        }
+
+        public boolean isEmpty() {
+            return jObjects.isEmpty();
+        }
+
+        public void addAll(Collection<IJObject> jObjectCollection) {
+            jObjects.addAll(jObjectCollection);
+        }
+
+        public void clear() {
+            jObjects.clear();
+        }
+
+        public IJObject getElement(int index) {
+            return jObjects.get(index);
+        }
+
+        public int size() {
+            return jObjects.size();
+        }
+
+    }
+
+    public static final class JRecord implements IJObject {
+
+        private AMutableRecord value;
+        private ARecordType recordType;
+        private List<IJObject> fields;
+        private List<String> fieldNames;
+        private List<IAType> fieldTypes;
+        private int numFieldsAdded = 0;
+        private List<Boolean> openField;
+
+        public JRecord(ARecordType recordType) {
+            this.recordType = recordType;
+            this.fields = new ArrayList<IJObject>();
+            initFieldInfo();
+        }
+
+        public JRecord(ARecordType recordType, IJObject[] fields) {
+            this.recordType = recordType;
+            this.fields = new ArrayList<IJObject>();
+            for (IJObject jObject : fields) {
+                this.fields.add(jObject);
+            }
+            initFieldInfo();
+        }
+
+        public JRecord(String[] fieldNames, IJObject[] fields) throws AsterixException {
+            this.recordType = getARecordType(fieldNames, fields);
+            this.fields = new ArrayList<IJObject>();
+            for (IJObject jObject : fields) {
+                this.fields.add(jObject);
+            }
+            initFieldInfo();
+        }
+
+        private ARecordType getARecordType(String[] fieldNames, IJObject[] fields) throws AsterixException {
+            IAType[] fieldTypes = new IAType[fields.length];
+            int index = 0;
+            for (IJObject jObj : fields) {
+                fieldTypes[index++] = jObj.getIAObject().getType();
+            }
+            ARecordType recordType = new ARecordType(null, fieldNames, fieldTypes, false);
+            return recordType;
+        }
+
+        private void initFieldInfo() {
+            this.openField = new ArrayList<Boolean>();
+            fieldNames = new ArrayList<String>();
+            for (String name : recordType.getFieldNames()) {
+                fieldNames.add(name);
+                openField.add(false);
+            }
+            fieldTypes = new ArrayList<IAType>();
+            for (IAType type : recordType.getFieldTypes()) {
+                fieldTypes.add(type);
+            }
+
+        }
+
+        private IAObject[] getIAObjectArray(List<IJObject> fields) {
+            IAObject[] retValue = new IAObject[fields.size()];
+            int index = 0;
+            for (IJObject jObject : fields) {
+                retValue[index++] = getIAObject(jObject);
+            }
+            return retValue;
+        }
+
+        private IAObject getIAObject(IJObject jObject) {
+            IAObject retVal = null;
+            switch (jObject.getTypeTag()) {
+                case RECORD:
+                    ARecordType recType = ((JRecord) jObject).getRecordType();
+                    IAObject[] fields = new IAObject[((JRecord) jObject).getFields().size()];
+                    int index = 0;
+                    for (IJObject field : ((JRecord) jObject).getFields()) {
+                        fields[index++] = getIAObject(field);
+                    }
+                    retVal = new AMutableRecord(recType, fields);
+                default:
+                    retVal = jObject.getIAObject();
+                    break;
+            }
+            return retVal;
+        }
+
+        public void addField(String fieldName, IJObject fieldValue) {
+            int pos = getFieldPosByName(fieldName);
+            if (pos >= 0) {
+                throw new IllegalArgumentException("field already defined");
+            }
+            numFieldsAdded++;
+            fields.add(fieldValue);
+            fieldNames.add(fieldName);
+            fieldTypes.add(fieldValue.getIAObject().getType());
+            openField.add(true);
+        }
+
+        public IJObject getValueByName(String fieldName) throws AsterixException, IOException {
+            int fieldPos = getFieldPosByName(fieldName);
+            if (fieldPos < 0) {
+                throw new AsterixException("unknown field: " + fieldName);
+            }
+            return fields.get(fieldPos);
+        }
+
+        public void setValueAtPos(int pos, IJObject jtype) {
+            fields.set(pos, jtype);
+        }
+
+        public void setValue(AMutableRecord mutableRecord) {
+            this.value = mutableRecord;
+            this.recordType = mutableRecord.getType();
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return recordType.getTypeTag();
+        }
+
+        public void setField(String fieldName, IJObject fieldValue) {
+            int pos = getFieldPosByName(fieldName);
+            fields.set(pos, fieldValue);
+            if (value != null) {
+                value.setValueAtPos(pos, fieldValue.getIAObject());
+            }
+        }
+
+        private int getFieldPosByName(String fieldName) {
+            int index = 0;
+            for (String name : fieldNames) {
+                if (name.equals(fieldName)) {
+                    return index;
+                }
+                index++;
+            }
+            return -1;
+        }
+
+        public ARecordType getRecordType() {
+            return recordType;
+        }
+
+        public List<IJObject> getFields() {
+            return fields;
+        }
+
+        @Override
+        public IAObject getIAObject() {
+            if (value == null || numFieldsAdded > 0) {
+                value = new AMutableRecord(recordType, getIAObjectArray(fields));
+            }
+            return value;
+        }
+
+        public void close() {
+            if (numFieldsAdded > 0) {
+                int totalFields = fieldNames.size();
+                for (int i = 0; i < numFieldsAdded; i++) {
+                    fieldNames.remove(totalFields - 1 - i);
+                    fieldTypes.remove(totalFields - 1 - i);
+                    fields.remove(totalFields - 1 - i);
+                }
+                numFieldsAdded = 0;
+            }
+        }
+
+        public List<Boolean> getOpenField() {
+            return openField;
+        }
+
+        public List<String> getFieldNames() {
+            return fieldNames;
+        }
+
+        public List<IAType> getFieldTypes() {
+            return fieldTypes;
+        }
+
+    }
+
+    public static class ByteArrayAccessibleInputStream extends ByteArrayInputStream {
+
+        public ByteArrayAccessibleInputStream(byte[] buf, int offset, int length) {
+            super(buf, offset, length);
+        }
+
+        public void setContent(byte[] buf, int offset, int length) {
+            this.buf = buf;
+            this.pos = offset;
+            this.count = Math.min(offset + length, buf.length);
+            this.mark = offset;
+        }
+
+        public byte[] getArray() {
+            return buf;
+        }
+
+        public int getPosition() {
+            return pos;
+        }
+
+        public int getCount() {
+            return count;
+        }
+
+    }
+
+    public static class ByteArrayAccessibleDataInputStream extends DataInputStream {
+
+        public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
+            super(in);
+        }
+
+        public ByteArrayAccessibleInputStream getInputStream() {
+            return (ByteArrayAccessibleInputStream) in;
+        }
+
+    }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java
new file mode 100644
index 0000000..3b686f6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java
@@ -0,0 +1,12 @@
+package edu.uci.ics.asterix.external.library.java;
+
+public enum JTypeTag {
+
+    INT,
+    STRING,
+    LONG,
+    DOUBLE,
+    FLOAT,
+    LIST,
+    OBJECT
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java
deleted file mode 100644
index 87e8478..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java
+++ /dev/null
@@ -1,43 +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.feed.managed.adapter;
-
-import java.util.Map;
-
-/**
- * Interface implemented by an adapter that can be controlled or managed by external
- * commands (stop,alter)
- */
-public interface IManagedFeedAdapter {
-
-    /**
-     * Discontinue the ingestion of data and end the feed.
-     * 
-     * @throws Exception
-     */
-    public void stop();
-
-    /**
-     * Modify the adapter configuration parameters. This method is called
-     * when the configuration parameters need to be modified while the adapter
-     * is ingesting data in an active feed.
-     * 
-     * @param properties
-     *            A HashMap containing the set of configuration parameters
-     *            that need to be altered.
-     */
-    public void alter(Map<String, String> properties);
-
-}
diff --git a/asterix-external-data/src/main/resources/feed_policy/basic.pol b/asterix-external-data/src/main/resources/feed_policy/basic.pol
new file mode 100644
index 0000000..8aed9f8
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/basic.pol
@@ -0,0 +1,8 @@
+feed.policy.name=Basic
+
+software.failure.persist.exception=false
+software.failure.continue.on.failure=false
+hardware.failure.auto.restart=false
+
+statistics.collect=false
+elastic=false
diff --git a/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol b/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol
new file mode 100644
index 0000000..efca65e
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol
@@ -0,0 +1,11 @@
+feed.policy.name=Basic_Monitored
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=false
+hardware.failure.auto.restart=false
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=false
diff --git a/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol b/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol
new file mode 100644
index 0000000..0a1da08
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol
@@ -0,0 +1,17 @@
+feed.policy.name=Mission_Critical
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=true
+hardware.failure.auto.restart=true
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=true
+criterion.ingestion.rate.percent=true
+criterion.ingestion.rate.percent.delta=50
+
+criterion.ingestion.rate.absolute=true
+criterion.ingestion.rate.absolute.delta=2000
+criterion.ingestion.rate.absolute.unit=sec
diff --git a/asterix-external-data/src/main/resources/feed_policy/twitter.pol b/asterix-external-data/src/main/resources/feed_policy/twitter.pol
new file mode 100644
index 0000000..5fa507f
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/twitter.pol
@@ -0,0 +1,17 @@
+feed.policy.name=Twitter
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=true
+hardware.failure.auto.restart=false
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=true
+criterion.ingestion.rate.percent=true
+criterion.ingestion.rate.percent.delta=50
+
+criterion.ingestion.rate.absolute=true
+criterion.ingestion.rate.absolute.delta=2000
+criterion.ingestion.rate.absolute.unit=sec
diff --git a/asterix-external-data/src/main/resources/schema/library.xsd b/asterix-external-data/src/main/resources/schema/library.xsd
new file mode 100644
index 0000000..3dc4659
--- /dev/null
+++ b/asterix-external-data/src/main/resources/schema/library.xsd
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:lib="library" targetNamespace="library" elementFormDefault="qualified">
+
+<!-- definition of simple types --> 
+<xs:element name="language" type="xs:string"/>
+<xs:element name="name" type="xs:string"/>
+<xs:element name="arguments" type="xs:string"/>
+<xs:element name="return_type" type="xs:string"/>
+<xs:element name="function_type" type="xs:string"/>
+<xs:element name="definition" type="xs:string"/>
+
+<!-- definition of complex elements -->
+<xs:element name="function">
+  <xs:complexType>
+    <xs:sequence>
+      <xs:element ref="lib:name"/>
+      <xs:element ref="lib:function_type"/>
+      <xs:element ref="lib:arguments"/>
+      <xs:element ref="lib:return_type"/>
+      <xs:element ref="lib:definition"/>
+    </xs:sequence>
+  </xs:complexType>
+</xs:element>
+
+<xs:element name="functions">
+  <xs:complexType>
+    <xs:sequence>
+      <xs:element ref="lib:function" maxOccurs="unbounded"/>
+    </xs:sequence>
+  </xs:complexType>
+</xs:element>
+
+<xs:element name="library">
+  <xs:complexType>
+    <xs:sequence>
+      <xs:element ref="lib:language"/>
+      <xs:element ref="lib:functions" minOccurs="0"/>
+    </xs:sequence>
+  </xs:complexType>
+</xs:element>
+
+</xs:schema>     
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
similarity index 64%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
index 98d72f4..f095321 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
@@ -3,28 +3,25 @@
  * 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.installer.service;
+package edu.uci.ics.asterix.external.library;
 
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
 
-    public static ServiceProvider INSTANCE = new ServiceProvider();
-    private static ILookupService lookupService = new ZooKeeperService();
-    
-    private ServiceProvider() {
+public class AllTypesFactory implements IFunctionFactory {
 
+    @Override
+    public IExternalFunction getExternalFunction() {
+        return new AllTypesFunction();
     }
 
-    public ILookupService getLookupService() {
-        return lookupService;
-    }
- 
 }
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java
new file mode 100644
index 0000000..c5063db
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java
@@ -0,0 +1,115 @@
+/*
+ * 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.external.library;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class AllTypesFunction implements IExternalScalarFunction {
+
+	private JOrderedList newFieldList;
+
+	@Override
+	public void initialize(IFunctionHelper functionHelper) {
+		newFieldList = new JOrderedList(functionHelper.getObject(JTypeTag.INT));
+	}
+
+	@Override
+	public void deinitialize() {
+	}
+
+	@Override
+	public void evaluate(IFunctionHelper functionHelper) throws Exception {
+		newFieldList.clear();
+		JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+		JInt id = (JInt) inputRecord.getValueByName("id");
+		JString name = (JString) inputRecord.getValueByName("name");
+		JFloat age = (JFloat) inputRecord.getValueByName("age");
+		JDouble salary = (JDouble) inputRecord.getValueByName("salary");
+		JBoolean married = (JBoolean) inputRecord.getValueByName("married");
+		JUnorderedList interest = (JUnorderedList) inputRecord
+				.getValueByName("interests");
+		JOrderedList children = (JOrderedList) inputRecord
+				.getValueByName("children");
+		JRecord address = (JRecord) inputRecord.getValueByName("address");
+		JDate dob = (JDate) inputRecord.getValueByName("dob");
+		JTime time = (JTime) inputRecord.getValueByName("time");
+		JDateTime dateTime = (JDateTime) inputRecord.getValueByName("datetime");
+		JDuration duration = (JDuration) inputRecord.getValueByName("duration");
+		JPoint location2d = (JPoint) inputRecord.getValueByName("location2d");
+		JPoint3D location3d = (JPoint3D) inputRecord
+				.getValueByName("location3d");
+		JLine line = (JLine) inputRecord.getValueByName("line");
+		JPolygon polygon = (JPolygon) inputRecord.getValueByName("polygon");
+		JCircle circle = (JCircle) inputRecord.getValueByName("circle");
+
+		JRecord result = (JRecord) functionHelper.getResultObject();
+		result.setField("id", id);
+		result.setField("name", name);
+		result.setField("age", age);
+		result.setField("salary", salary);
+		result.setField("married", married);
+		result.setField("interests", interest);
+		result.setField("children", children);
+		JInt zipCode = (JInt) functionHelper.getObject(JTypeTag.INT);
+		zipCode.setValue(92841);
+		address.addField("Zipcode", zipCode);
+		result.setField("address", address);
+		result.setField("dob", dob);
+		result.setField("time", time);
+		result.setField("datetime", dateTime);
+		result.setField("duration", duration);
+		result.setField("location2d", location2d);
+		result.setField("location3d", location3d);
+		result.setField("line", line);
+		result.setField("polygon", polygon);
+		result.setField("circle", circle);
+
+		JString newFieldString = (JString) functionHelper
+				.getObject(JTypeTag.STRING);
+		newFieldString.setValue("processed");
+		result.addField("status", newFieldString);
+
+		/*
+		 * JString element = (JString)
+		 * functionHelper.getObject(JTypeTag.STRING); element.setValue("raman");
+		 * newFieldList.add(element); result.addField("mylist", newFieldList);
+		 */
+
+		JString newFieldString2 = (JString) functionHelper
+				.getObject(JTypeTag.STRING);
+		newFieldString2.setValue("this is working");
+		result.addField("working", newFieldString);
+		functionHelper.setResult(result);
+	}
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
similarity index 63%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
index 98d72f4..872b542 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
@@ -3,28 +3,25 @@
  * 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.installer.service;
+package edu.uci.ics.asterix.external.library;
 
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
 
-    public static ServiceProvider INSTANCE = new ServiceProvider();
-    private static ILookupService lookupService = new ZooKeeperService();
-    
-    private ServiceProvider() {
+public class CapitalFinderFactory implements IFunctionFactory {
 
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new CapitalFinderFunction();
     }
 
-    public ILookupService getLookupService() {
-        return lookupService;
-    }
- 
 }
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java
new file mode 100644
index 0000000..7953b62
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java
@@ -0,0 +1,57 @@
+/*
+ * 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.external.library;
+
+import java.io.InputStream;
+import java.util.Properties;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class CapitalFinderFunction implements IExternalScalarFunction {
+
+    private static Properties capitalList;
+    private static final String NOT_FOUND = "NOT_FOUND";
+    private JString capital;
+
+    @Override
+    public void deinitialize() {
+        System.out.println(" De Initialized");
+    }
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        JString country = ((JString) functionHelper.getArgument(0));
+        JRecord record = (JRecord) functionHelper.getResultObject();
+        //     ((JString) record.getValueByName("country")).setValue(country.getValue());
+        String capitalCity = capitalList.getProperty(country.getValue(), NOT_FOUND);
+        //      ((JString) record.getValueByName("capital")).setValue(capitalCity);
+        capital.setValue(capitalCity);
+
+        record.setField("country", country);
+        record.setField("capital", capital);
+        functionHelper.setResult(record);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) throws Exception {
+        InputStream in = CapitalFinderFunction.class.getClassLoader().getResourceAsStream("data/countriesCapitals.txt");
+        capitalList = new Properties();
+        capitalList.load(in);
+        capital = (JString) functionHelper.getObject(JTypeTag.STRING);
+    }
+
+}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
similarity index 72%
copy from asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
index af46e63..d15d661 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
@@ -3,21 +3,23 @@
  * 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.event.xml;
+package edu.uci.ics.asterix.external.library;
 
-public class PatternParser {
 
-	public static void parsePattern(String path){
-		
-	}
+public class EchoDelayFactory implements IFunctionFactory {
+
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new EchoDelayFunction();
+    }
+
 }
-
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java
new file mode 100644
index 0000000..5f9be77
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java
@@ -0,0 +1,46 @@
+/*
+ * 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.external.library;
+
+import java.util.Random;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+
+public class EchoDelayFunction implements IExternalScalarFunction {
+
+    private Random rand = new Random();
+    private long sleepIntervalMin;
+    private long sleepIntervalMax;
+    private int range;
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) {
+        sleepIntervalMin = 50;
+        sleepIntervalMax = 100;
+        range = (new Long(sleepIntervalMax - sleepIntervalMin)).intValue();
+    }
+
+    @Override
+    public void deinitialize() {
+    }
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+        long sleepInterval = rand.nextInt(range);
+     //   Thread.sleep(5);
+        functionHelper.setResult(inputRecord);
+    }
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
similarity index 63%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
index 98d72f4..d868f20 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
@@ -3,28 +3,25 @@
  * 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.installer.service;
+package edu.uci.ics.asterix.external.library;
 
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
 
-    public static ServiceProvider INSTANCE = new ServiceProvider();
-    private static ILookupService lookupService = new ZooKeeperService();
-    
-    private ServiceProvider() {
+public class ParseTweetFactory implements IFunctionFactory {
 
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new ParseTweetFunction();
     }
 
-    public ILookupService getLookupService() {
-        return lookupService;
-    }
- 
 }
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java
new file mode 100644
index 0000000..de13e59
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java
@@ -0,0 +1,60 @@
+/*
+ * 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.external.library;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class ParseTweetFunction implements IExternalScalarFunction {
+
+    private JOrderedList list = null;
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) {
+        list = new JOrderedList(functionHelper.getObject(JTypeTag.STRING));
+    }
+
+    @Override
+    public void deinitialize() {
+    }
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        list.clear();
+        JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+        JString id = (JString) inputRecord.getValueByName("id");
+        JString text = (JString) inputRecord.getValueByName("text");
+
+        String[] tokens = text.getValue().split(" ");
+        for (String tk : tokens) {
+            if (tk.startsWith("#")) {
+                JString newField = (JString) functionHelper.getObject(JTypeTag.STRING);
+                newField.setValue(tk);
+                list.add(newField);
+            }
+        }
+        JRecord result = (JRecord) functionHelper.getResultObject();
+        result.setField("id", id);
+        result.setField("username", inputRecord.getValueByName("username"));
+        result.setField("location", inputRecord.getValueByName("location"));
+        result.setField("text", text);
+        result.setField("timestamp", inputRecord.getValueByName("timestamp"));
+        result.setField("topics", list);
+        functionHelper.setResult(result);
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
similarity index 64%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
index dcef2c8..eabc6b5 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
@@ -3,26 +3,25 @@
  * 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.external.feed.lifecycle;
+package edu.uci.ics.asterix.external.library;
+        
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
 
-import java.io.Serializable;
+public class SumFactory implements IFunctionFactory {
 
-public interface IFeedMessage extends Serializable {
-
-    public enum MessageType {
-        STOP,
-        ALTER,
-    }
-
-    public MessageType getMessageType();
+	@Override
+	public IExternalScalarFunction getExternalFunction() {
+		return new SumFunction();
+	}
 
 }
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java
new file mode 100644
index 0000000..0a4eca6
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java
@@ -0,0 +1,43 @@
+/*
+ * 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.external.library;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+
+public class SumFunction implements IExternalScalarFunction {
+
+    private JInt result;
+
+    @Override
+    public void deinitialize() {
+        System.out.println(" De Initialized");
+    }
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        int arg0 = ((JInt) functionHelper.getArgument(0)).getValue();
+        int arg1 = ((JInt) functionHelper.getArgument(1)).getValue();
+        result.setValue(arg0 + arg1);
+        functionHelper.setResult(result);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) {
+        result = (JInt) functionHelper.getResultObject();
+    }
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
similarity index 64%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
index 98d72f4..2c5f607 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
@@ -3,28 +3,25 @@
  * 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.installer.service;
+package edu.uci.ics.asterix.external.library;
 
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
 
-    public static ServiceProvider INSTANCE = new ServiceProvider();
-    private static ILookupService lookupService = new ZooKeeperService();
-    
-    private ServiceProvider() {
+public class UpperCaseFactory implements IFunctionFactory {
 
+    @Override
+    public IExternalFunction getExternalFunction() {
+        return new UpperCaseFunction();
     }
 
-    public ILookupService getLookupService() {
-        return lookupService;
-    }
- 
 }
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java
new file mode 100644
index 0000000..e3d1c9c
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java
@@ -0,0 +1,67 @@
+/*
+ * 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.external.library;
+
+import java.util.Random;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+
+/**
+ * Accepts an input record of type Open{ id: int32, text: string }
+ * 
+ * Converts the text field into upper case and appends an additional field -
+ * "substring" with value as a random substring of the text field.
+ * 
+ * Return Type Open{ id: int32, text: string }
+ * 
+ */
+public class UpperCaseFunction implements IExternalScalarFunction {
+
+	private Random random;
+
+	@Override
+	public void initialize(IFunctionHelper functionHelper) {
+		random = new Random();
+	}
+
+	@Override
+	public void deinitialize() {
+	}
+
+	@Override
+	public void evaluate(IFunctionHelper functionHelper) throws Exception {
+		JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+		JInt id = (JInt) inputRecord.getValueByName("id");
+		id.setValue(id.getValue() * -1); // for maintaining uniqueness
+											// constraint in the case when
+											// output is re-inserted into source
+											// dataset
+		JString text = (JString) inputRecord.getValueByName("text");
+		text.setValue(text.getValue().toUpperCase());
+		JRecord result = (JRecord) functionHelper.getResultObject();
+		result.setField("id", id);
+		result.setField("text", text);
+		JString newField = (JString) functionHelper.getObject(JTypeTag.STRING);
+		newField.setValue(text.getValue().substring(
+				random.nextInt(text.getValue().length())));
+		result.addField("substring", newField);
+		functionHelper.setResult(result);
+	}
+}
diff --git a/asterix-external-data/src/test/resources/data/countriesCapitals.txt b/asterix-external-data/src/test/resources/data/countriesCapitals.txt
new file mode 100644
index 0000000..def17ed
--- /dev/null
+++ b/asterix-external-data/src/test/resources/data/countriesCapitals.txt
@@ -0,0 +1,189 @@
+United\ Arab\ Emirates=Abu Dhabi
+Nigeria=Abuja
+Ghana=Accra
+Ethiopia=Addis Ababa
+Algeria=Algiers
+Niue=Alofi
+Jordan=Amman
+Netherlands=Amsterdam 
+Andorra=Andorra la Vella
+Turkey=Ankara
+Madagascar=Antananarivo
+Samoa=Apia
+Turkmenistan=Ashgabat
+Eritrea=Asmara
+Kazakhstan=Astana
+Paraguay=Asunción
+Greece=Athens
+Iraq=Baghdad
+Azerbaijan=Baku
+Mali=Bamako
+Brunei=Bandar Seri Begawan
+Thailand=Bangkok
+Gambia=Banjul
+China=Beijing
+Lebanon=Beirut
+Serbia=Belgrade
+Belize=Belmopan
+Germany=Berlin
+Switzerland=Bern
+Kyrgyzstan=Bishkek
+Guinea-Bissau=Bissau
+Colombia=Bogotá
+Brazil=Brasília
+Slovakia=Bratislava
+Barbados=Bridgetown
+Belgium=Brussels
+Romania=Bucharest
+Hungary=Budapest
+Argentina=Buenos Aires
+Burundi=Bujumbura
+Egypt=Cairo
+Australia=Canberra
+Venezuela=Caracas
+Wales=Cardiff
+Moldova=Chisinau
+Guinea=Conakry
+Denmark=Copenhagen
+Senegal=Dakar
+Syria=Damascus
+Bangladesh=Dhaka
+Djibouti=Djibouti
+Qatar=Doha
+Ireland=Dublin
+Tajikistan=Dushanbe
+Scotland=Edinburgh
+Botswana=Gaborone
+Guyana=Georgetown
+Gibraltar=Gibraltar
+Guatemala=Guatemala City
+Guam=Hagåtña
+Bermuda=Hamilton
+Vietnam=Hanoi
+Zimbabwe=Harare
+Cuba=Havana
+Finland=Helsinki
+Pakistan=Islamabad
+Indonesia=Jakarta
+Afghanistan=Kabul
+Uganda=Kampala
+Nepal=Kathmandu
+Sudan=Khartoum
+Ukraine=Kiev
+Rwanda=Kigali
+Jamaica=Kingston
+Malaysia=Kuala Lumpur
+Kuwait=Kuwait City
+Gabon=Libreville
+Malawi=Lilongwe
+Peru=Lima
+Portugal=Lisbon
+Slovenia=Ljubljana
+Togo=Lome
+England=London
+Angola=Luanda
+Zambia=Lusaka
+Luxembourg=Luxembourg
+Spain=Madrid
+Marshall Islands=Majuro
+Equatorial Guinea=Malabo
+Maldives=Malé
+Nicaragua=Managua
+Bahrain=Manama
+Philippines=Manila
+Mozambique=Maputo
+Saint Martin=Marigot
+Lesotho=Maseru
+Wallis and Futuna=Mata-Utu
+Palau=Melekeok
+Mexico=Mexico City
+Belarus=Minsk
+Somalia=Mogadishu
+Monaco=Monaco
+Liberia=Monrovia
+Uruguay=Montevideo
+Comoros=Moroni
+Russia=Moscow
+Oman=Muscat
+Kenya=Nairobi
+Bahamas=Nassau
+Myanmar=Naypyidaw
+Chad=NDjamena
+India=New Delhi
+Niger=Niamey
+Cyprus=Nicosia
+Mauritania=Nouakchott
+Greenland=Nuuk
+Aruba=Oranjestad
+Norway=Oslo
+Canada=Ottawa
+Panama=Panama City
+Suriname=Paramaribo
+France=Paris
+Cambodia=Phnom Penh
+Montserrat=Brades Estate (de facto)
+Montenegro=Podgorica
+Mauritius=Port Louis
+Vanuatu=Port Vila
+Haiti=Port-au-Prince
+Benin=Cotonou (de facto)
+Czech Republic=Prague
+Cape Verde=Praia
+South\ Africa=Cape Town
+North\ Korea=Pyongyang
+Ecuador=Quito
+Morocco=Rabat
+Iceland=Reykjavík
+Latvia=Riga
+Saudi\ Arabia=Riyadh
+Italy=Rome
+Dominica=Roseau
+Costa\ Rica=San José
+Puerto\ Rico=San Juan
+San\ Marino=San Marino
+El\ Salvador=San Salvador
+Yemen=Sanaa
+Chile=Santiago
+Dominican\ Republic=Santo Domingo
+Bosnia\ and\ Herzegovina=Sarajevo
+South\ Korea=Seoul
+Singapore=Singapore
+Macedonia=Skopje
+Bulgaria=Sofia
+Sri\ Lanka=Colombo
+Grenada=St. George's
+Jersey=St. Helier
+Guernsey=St. Peter Port
+Sweden=Stockholm
+Bolivia=La Paz
+Abkhazia=Sukhumi
+Fiji=Suva
+Taiwan=Taipei
+Estonia=Tallinn
+Kiribati=Tarawa
+Uzbekistan=Tashkent
+Georgia=Tbilisi
+Honduras=Tegucigalpa
+Iran=Tehran
+Bhutan=Thimphu
+Albania=Tirana
+Transnistria=Tiraspol
+Japan=Tokyo
+Libya=Tripoli
+Tunisia=Tunis
+Mongolia=Ulan Bator
+Liechtenstein=Vaduz
+Malta=Valletta
+Anguilla=The Valley
+Vatican\ City=Vatican City
+Seychelles=Victoria
+Austria=Vienna
+Laos=Vientiane
+Lithuania=Vilnius
+Poland=Warsaw
+United\ States=Washington D.C.
+New\ Zealand=Wellington
+Namibia=Windhoek
+Nauru=Yaren (de facto)
+Armenia=Yerevan
+Croatia=Zagreb
diff --git a/asterix-external-data/src/test/resources/text_functions.xml b/asterix-external-data/src/test/resources/text_functions.xml
new file mode 100644
index 0000000..6d00029
--- /dev/null
+++ b/asterix-external-data/src/test/resources/text_functions.xml
@@ -0,0 +1,53 @@
+<library xmlns="library">
+	<language>JAVA</language>
+	<functions>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>parseTweet</name>
+			<arguments>TweetType</arguments>
+			<return_type>TweetType</return_type>
+			<definition>edu.uci.ics.asterix.external.library.ParseTweetFactory
+			</definition>
+		</function>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>mysum</name>
+			<arguments>AINT32,AINT32</arguments>
+			<return_type>AINT32</return_type>
+			<definition>edu.uci.ics.asterix.external.library.SumFactory
+			</definition>
+		</function>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>getCapital</name>
+			<arguments>ASTRING</arguments>
+			<return_type>CountryCapitalType</return_type>
+			<definition>edu.uci.ics.asterix.external.library.CapitalFinderFactory
+			</definition>
+		</function>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>toUpper</name>
+			<arguments>TextType</arguments>
+			<return_type>TextType</return_type>
+			<definition>edu.uci.ics.asterix.external.library.UpperCaseFactory
+			</definition>
+		</function>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>allTypes</name>
+			<arguments>AllType</arguments>
+			<return_type>AllType</return_type>
+			<definition>edu.uci.ics.asterix.external.library.AllTypesFactory
+			</definition>
+		</function>
+		<function>
+			<function_type>SCALAR</function_type>
+			<name>echoDelay</name>
+			<arguments>TweetMessageType</arguments>
+			<return_type>TweetMessageType</return_type>
+			<definition>edu.uci.ics.asterix.external.library.EchoDelayFactory
+			</definition>
+		</function>
+	</functions>
+</library>
diff --git a/asterix-installer/pom.xml b/asterix-installer/pom.xml
index 242f873..ad515ec 100644
--- a/asterix-installer/pom.xml
+++ b/asterix-installer/pom.xml
@@ -1,25 +1,21 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-          <artifactId>asterix</artifactId>
-          <groupId>edu.uci.ics.asterix</groupId>
-          <version>0.8.1-SNAPSHOT</version>
-  </parent>
-  <artifactId>asterix-installer</artifactId>
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<artifactId>asterix</artifactId>
+		<groupId>edu.uci.ics.asterix</groupId>
+		<version>0.8.1-SNAPSHOT</version>
+	</parent>
+	<artifactId>asterix-installer</artifactId>
 	<properties>
 		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
 	</properties>
@@ -31,11 +27,10 @@
 				<artifactId>maven-compiler-plugin</artifactId>
 				<version>2.0.2</version>
 				<configuration>
-					<source>1.6</source>
-					<target>1.6</target>
+					<source>1.7</source>
+					<target>1.7</target>
 				</configuration>
 			</plugin>
-
 			<plugin>
 				<groupId>org.jvnet.jaxb2.maven2</groupId>
 				<artifactId>maven-jaxb2-plugin</artifactId>
@@ -76,10 +71,6 @@
 							<goal>generate</goal>
 						</goals>
 						<configuration>
-							<args>
-								<arg>-Xsetters</arg>
-								<arg>-Xvalue-constructor</arg>
-							</args>
 							<schemaDirectory>src/main/resources/schema</schemaDirectory>
 							<schemaIncludes>
 								<include>cluster.xsd</include>
@@ -174,16 +165,9 @@
 			<groupId>edu.uci.ics.asterix</groupId>
 			<artifactId>asterix-common</artifactId>
 			<version>0.8.1-SNAPSHOT</version>
-			<type>jar</type>
-			<scope>compile</scope>
+			<type>test-jar</type>
+			<scope>test</scope>
 		</dependency>
-                <dependency>
-                        <groupId>edu.uci.ics.asterix</groupId>
-                        <artifactId>asterix-common</artifactId>
-                        <version>0.8.1-SNAPSHOT</version>
-                        <type>test-jar</type>
-                        <scope>test</scope>
-                </dependency>
 		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
 			<artifactId>asterix-server</artifactId>
@@ -191,12 +175,11 @@
 			<type>zip</type>
 			<classifier>binary-assembly</classifier>
 		</dependency>
-                <dependency>
-                        <groupId>edu.uci.ics.asterix</groupId>
-                        <artifactId>asterix-test-framework</artifactId>
-                        <version>0.8.1-SNAPSHOT</version>
-                        <scope>test</scope>
-                </dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-test-framework</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
 	</dependencies>
-
 </project>
diff --git a/asterix-installer/src/main/assembly/binary-assembly.xml b/asterix-installer/src/main/assembly/binary-assembly.xml
index d42c5ae..930f686 100644
--- a/asterix-installer/src/main/assembly/binary-assembly.xml
+++ b/asterix-installer/src/main/assembly/binary-assembly.xml
@@ -13,112 +13,108 @@
  ! limitations under the License.
  !-->
 <assembly>
-  <id>binary-assembly</id>
-  <formats>
-    <format>zip</format>
-    <format>dir</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>src/main/resources/conf</directory>
-      <outputDirectory>conf</outputDirectory>
-    </fileSet>
-    <fileSet>
-      <directory>src/main/resources/clusters</directory>
-      <outputDirectory>clusters</outputDirectory>
-    </fileSet>
-    <fileSet>
-      <directory>src/main/resources/zookeeper</directory>
-      <fileMode>0755</fileMode>
-      <outputDirectory>.installer/zookeeper/bin</outputDirectory>
-    </fileSet>
-    <fileSet>
-      <directory>src/main/resources/scripts</directory>
-      <fileMode>0755</fileMode>
-      <includes>
-        <include>managix</include>
-      </includes>
-      <outputDirectory>bin</outputDirectory>
-    </fileSet>
-    <fileSet>
-      <directory>src/main/resources/scripts</directory>
-      <fileMode>0755</fileMode>
-      <excludes>
-        <exclude>managix</exclude>
-      </excludes>
-      <outputDirectory>.installer/scripts</outputDirectory>
-    </fileSet>
-    <fileSet>
-      <directory>src/main/resources/hadoop-0.20.2</directory>
-      <outputDirectory>.installer/hadoop-0.20.2</outputDirectory>
-      <fileMode>0755</fileMode>
-    </fileSet>
-    <fileSet>
-      <directory>target</directory>
-      <outputDirectory>lib</outputDirectory>
-      <includes>
-        <include>*.jar</include>
-      </includes>
-    </fileSet>
-    <fileSet>
-      <directory>../asterix-events/src/main/resources/events</directory>
-      <outputDirectory>.installer/eventrix/events</outputDirectory>
-      <fileMode>0755</fileMode>
-    </fileSet>
-    <fileSet>
-      <directory>../asterix-events/src/main/resources/scripts</directory>
-      <outputDirectory>.installer/eventrix/scripts</outputDirectory>
-    </fileSet>
-  </fileSets>
-  <dependencySets>
-    <dependencySet>
-      <includes>
-         <include>log4j:log4j</include>
-         <include>edu.uci.ics.asterix:asterix-events</include>
-         <include>edu.uci.ics.asterix:asterix-common</include>
-         <include>org.apache.zookeeper:zookeeper</include>
-         <include>args4j:args4j</include>
-         <include>log4j:log4j</include>
-         <include>commons-io:commons-io</include>
-         <include>org.slf4j:slf4j-api</include>
-         <include>org.slf4j:slf4j-log4j12</include>
-       </includes>
-       <unpack>false</unpack>
-       <outputDirectory>lib</outputDirectory>
-    </dependencySet>
-    <dependencySet>
-      <includes>
-        <include>org.apache.hadoop:hadoop-core</include>
-        <include>commons-cli:commons-cli</include>
-        <include>commons-logging:commons-logging</include>
-      </includes>
-      <unpack>false</unpack>
-      <outputDirectory>.installer/hadoop-0.20.2/lib</outputDirectory>
-    </dependencySet>
-    <dependencySet>
-      <includes>
-        <include>org.apache.zookeeper:zookeeper</include>
-        <include>log4j:log4j</include>
-        <include>org.slf4j:slf4j-api</include>
-      </includes>
-      <unpack>false</unpack>
-      <outputDirectory>.installer/zookeeper/lib</outputDirectory>
-    </dependencySet>
-    <dependencySet>
-      <outputDirectory>asterix</outputDirectory>
-      <includes>
-        <include>asterix-server*</include>
-      </includes>
-      <useTransitiveDependencies>false</useTransitiveDependencies>
-    </dependencySet>
-    <dependencySet>
-      <outputDirectory>.installer/eventrix</outputDirectory>
-      <includes>
-        <include>asterix-events*</include>
-      </includes>
-      <unpack>false</unpack>
-      <useTransitiveDependencies>false</useTransitiveDependencies>
-    </dependencySet>
-  </dependencySets>
-</assembly>
+	<id>binary-assembly</id>
+	<formats>
+		<format>zip</format>
+		<format>dir</format>
+	</formats>
+	<includeBaseDirectory>false</includeBaseDirectory>
+	<fileSets>
+		<fileSet>
+			<directory>src/main/resources/conf</directory>
+			<outputDirectory>conf</outputDirectory>
+		</fileSet>
+		<fileSet>
+			<directory>src/main/resources/clusters</directory>
+			<outputDirectory>clusters</outputDirectory>
+		</fileSet>
+		<fileSet>
+			<directory>src/main/resources/zookeeper</directory>
+			<fileMode>0755</fileMode>
+			<outputDirectory>.installer/zookeeper/bin</outputDirectory>
+		</fileSet>
+		<fileSet>
+			<directory>src/main/resources/scripts</directory>
+			<fileMode>0755</fileMode>
+			<includes>
+				<include>managix</include>
+			</includes>
+			<outputDirectory>bin</outputDirectory>
+		</fileSet>
+		<fileSet>
+			<directory>src/main/resources/scripts</directory>
+			<fileMode>0755</fileMode>
+			<excludes>
+				<exclude>managix</exclude>
+			</excludes>
+			<outputDirectory>.installer/scripts</outputDirectory>
+		</fileSet>
+		<fileSet>
+			<directory>src/main/resources/hadoop-0.20.2</directory>
+			<outputDirectory>.installer/hadoop-0.20.2</outputDirectory>
+			<fileMode>0755</fileMode>
+		</fileSet>
+		<fileSet>
+			<directory>target</directory>
+			<outputDirectory>lib</outputDirectory>
+			<includes>
+				<include>*.jar</include>
+			</includes>
+		</fileSet>
+		<fileSet>
+			<directory>../asterix-events/src/main/resources/events</directory>
+			<outputDirectory>.installer/events</outputDirectory>
+			<fileMode>0755</fileMode>
+		</fileSet>
+	</fileSets>
+	<dependencySets>
+		<dependencySet>
+			<includes>
+				<include>log4j:log4j</include>
+				<include>edu.uci.ics.asterix:asterix-events</include>
+				<include>edu.uci.ics.asterix:asterix-common</include>
+				<include>org.apache.zookeeper:zookeeper</include>
+				<include>args4j:args4j</include>
+				<include>log4j:log4j</include>
+				<include>commons-io:commons-io</include>
+				<include>org.slf4j:slf4j-api</include>
+				<include>org.slf4j:slf4j-log4j12</include>
+			</includes>
+			<unpack>false</unpack>
+			<outputDirectory>lib</outputDirectory>
+		</dependencySet>
+		<dependencySet>
+			<includes>
+				<include>org.apache.hadoop:hadoop-core</include>
+				<include>commons-cli:commons-cli</include>
+				<include>commons-logging:commons-logging</include>
+			</includes>
+			<unpack>false</unpack>
+			<outputDirectory>.installer/hadoop-0.20.2/lib</outputDirectory>
+		</dependencySet>
+		<dependencySet>
+			<includes>
+				<include>org.apache.zookeeper:zookeeper</include>
+				<include>log4j:log4j</include>
+				<include>org.slf4j:slf4j-api</include>
+			</includes>
+			<unpack>false</unpack>
+			<outputDirectory>.installer/zookeeper/lib</outputDirectory>
+		</dependencySet>
+		<dependencySet>
+			<outputDirectory>asterix</outputDirectory>
+			<includes>
+				<include>asterix-server*</include>
+			</includes>
+			<useTransitiveDependencies>false</useTransitiveDependencies>
+		</dependencySet>
+		<dependencySet>
+			<outputDirectory>.installer/events</outputDirectory>
+			<includes>
+				<include>asterix-events*</include>
+			</includes>
+			<unpack>false</unpack>
+			<useTransitiveDependencies>false</useTransitiveDependencies>
+		</dependencySet>
+	</dependencySets>
+ </assembly>
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
index 8e4b2bc..4be8cd5 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
@@ -19,53 +19,60 @@
 import org.kohsuke.args4j.Option;
 
 import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
 import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class AlterCommand extends AbstractCommand {
 
-    @Override
-    protected void execCommand() throws Exception {
-        InstallerDriver.initConfig(true);
-        String instanceName = ((AlterConfig) config).name;
-        InstallerUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
-        ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
-        AsterixInstance instance = lookupService.getAsterixInstance(instanceName);
-        InstallerUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
-        AsterixConfiguration asterixConfiguration = InstallerUtil
-                .getAsterixConfiguration(((AlterConfig) config).confPath);
-        instance.setAsterixConfiguration(asterixConfiguration);
-        instance.setModifiedTimestamp(new Date());
-        lookupService.updateAsterixInstance(instance);
-        LOGGER.info("Altered configuration settings for Asterix instance: " + instanceName);
+	@Override
+	protected void execCommand() throws Exception {
+		InstallerDriver.initConfig(true);
+		String instanceName = ((AlterConfig) config).name;
+		AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName,
+				State.INACTIVE);
+		ILookupService lookupService = ServiceProvider.INSTANCE
+				.getLookupService();
+		AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(instanceName);
+		AsterixEventServiceUtil.createClusterProperties(instance.getCluster(),
+				instance.getAsterixConfiguration());
+		AsterixConfiguration asterixConfiguration = InstallerUtil
+				.getAsterixConfiguration(((AlterConfig) config).confPath);
+		instance.setAsterixConfiguration(asterixConfiguration);
+		instance.setModifiedTimestamp(new Date());
+		lookupService.updateAsterixInstance(instance);
+		LOGGER.info("Altered configuration settings for Asterix instance: "
+				+ instanceName);
 
-    }
+	}
 
-    @Override
-    protected CommandConfig getCommandConfig() {
-        return new AlterConfig();
-    }
+	@Override
+	protected CommandConfig getCommandConfig() {
+		return new AlterConfig();
+	}
 
-    @Override
-    protected String getUsageDescription() {
-        return "\nAlter the instance's configuration settings."
-                + "\nPrior to running this command, the instance is required to be INACTIVE state."
-                + "\nChanged configuration settings will be reflected when the instance is started."
-                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance.";
-    }
+	@Override
+	protected String getUsageDescription() {
+		return "\nAlter the instance's configuration settings."
+				+ "\nPrior to running this command, the instance is required to be INACTIVE state."
+				+ "\nChanged configuration settings will be reflected when the instance is started."
+				+ "\n\nAvailable arguments/options"
+				+ "\n-n name of the ASTERIX instance.";
+	}
 
 }
 
 class AlterConfig extends CommandConfig {
 
-    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
-    public String name;
+	@Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+	public String name;
 
-    @Option(name = "-a", required = true, usage = "Path to asterix instance configuration")
-    public String confPath;
+	@Option(name = "-a", required = true, usage = "Path to asterix instance configuration")
+	public String confPath;
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
index 91cbc86..3f30619 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
@@ -19,15 +19,16 @@
 
 import org.kohsuke.args4j.Option;
 
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.BackupInfo;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
 import edu.uci.ics.asterix.installer.schema.conf.Backup;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class BackupCommand extends AbstractCommand {
 
@@ -37,12 +38,12 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((BackupConfig) config).name;
-        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE);
         List<BackupInfo> backupInfo = instance.getBackupInfo();
-        PatternCreator pc = new PatternCreator();
-        Backup backupConf = InstallerDriver.getConfiguration().getBackup();
-        Patterns patterns = pc.getBackUpAsterixPattern(instance, backupConf);
-        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        Backup backupConf = AsterixEventService.getConfiguration().getBackup();
+        Patterns patterns = PatternCreator.INSTANCE.getBackUpAsterixPattern(instance, backupConf);
+        AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
         int backupId = backupInfo.size();
         BackupInfo binfo = new BackupInfo(backupId, new Date(), backupConf);
         backupInfo.add(binfo);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
index 230a945..45cc9f7 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
@@ -52,6 +52,12 @@
             case CONFIGURE:
                 cmd = new ConfigureCommand();
                 break;
+            case INSTALL:
+                cmd = new InstallCommand();
+                break;
+            case UNINSTALL:
+                cmd = new UninstallCommand();
+                break;
             case LOG:
                 cmd = new LogCommand();
                 break;
@@ -61,6 +67,17 @@
             case HELP:
                 cmd = new HelpCommand();
                 break;
+            case STOPNODE:
+                cmd = new StopNodeCommand();
+                break;
+            case STARTNODE:
+                cmd = new StartNodeCommand();
+                break;
+            case VERSION:
+                cmd = new VersionCommand();
+                break;
+            default:
+                break;
         }
         cmd.execute(args);
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
index 2803fee..63d2f33 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
@@ -16,20 +16,25 @@
 
 import java.io.File;
 
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
 import org.kohsuke.args4j.Option;
 
 import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
 import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
 import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class CreateCommand extends AbstractCommand {
 
@@ -46,29 +51,31 @@
             throw new Exception("Cannot create an Asterix instance.");
         }
         asterixInstanceName = ((CreateConfig) config).name;
-        InstallerUtil.validateAsterixInstanceNotExists(asterixInstanceName);
+        AsterixEventServiceUtil.validateAsterixInstanceNotExists(asterixInstanceName);
         CreateConfig createConfig = (CreateConfig) config;
         cluster = EventUtil.getCluster(createConfig.clusterPath);
+        cluster.setInstanceName(asterixInstanceName);
         asterixConfiguration = InstallerUtil.getAsterixConfiguration(createConfig.asterixConfPath);
-        AsterixInstance asterixInstance = InstallerUtil.createAsterixInstance(asterixInstanceName, cluster,
+        AsterixInstance asterixInstance = AsterixEventServiceUtil.createAsterixInstance(asterixInstanceName, cluster,
                 asterixConfiguration);
-        InstallerUtil.evaluateConflictWithOtherInstances(asterixInstance);
-        InstallerUtil.createAsterixZip(asterixInstance);
-        InstallerUtil.createClusterProperties(cluster, asterixConfiguration);
-        EventrixClient eventrixClient = InstallerUtil.getEventrixClient(cluster);
-        PatternCreator pc = new PatternCreator();
+        AsterixEventServiceUtil.evaluateConflictWithOtherInstances(asterixInstance);
+        AsterixEventServiceUtil.createAsterixZip(asterixInstance);
+        AsterixEventServiceUtil.createClusterProperties(cluster, asterixConfiguration);
+        AsterixEventServiceClient eventrixClient = AsterixEventService.getAsterixEventServiceClient(cluster, true,
+                false);
 
-        Patterns asterixBinarytrasnferPattern = pc.getAsterixBinaryTransferPattern(asterixInstanceName, cluster);
+        Patterns asterixBinarytrasnferPattern = PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
+                asterixInstanceName, cluster);
         eventrixClient.submit(asterixBinarytrasnferPattern);
 
-        Patterns patterns = pc.getStartAsterixPattern(asterixInstanceName, cluster);
+        Patterns patterns = PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, cluster);
         eventrixClient.submit(patterns);
 
         AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(asterixInstance);
         VerificationUtil.updateInstanceWithRuntimeDescription(asterixInstance, runtimeState, true);
         ServiceProvider.INSTANCE.getLookupService().writeAsterixInstance(asterixInstance);
-        InstallerUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator + InstallerDriver.ASTERIX_DIR
-                + File.separator + asterixInstanceName);
+        AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator
+                + InstallerDriver.ASTERIX_DIR + File.separator + asterixInstanceName);
         LOGGER.info(asterixInstance.getDescription(false));
     }
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
index 00a3de4..3938ea5 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
@@ -16,13 +16,14 @@
 
 import org.kohsuke.args4j.Option;
 
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class DeleteCommand extends AbstractCommand {
 
@@ -30,13 +31,13 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((DeleteConfig) config).name;
-        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
-        PatternCreator pc = new PatternCreator();
-        Patterns patterns = pc.createDeleteInstancePattern(instance);
-        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE);
+        Patterns patterns = PatternCreator.INSTANCE.createDeleteInstancePattern(instance);
+        AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
 
-        patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
-        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        patterns = PatternCreator.INSTANCE.createRemoveAsterixWorkingDirPattern(instance);
+        AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
         ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(asterixInstanceName);
         LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
index 58851cb..9010061 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
@@ -18,14 +18,14 @@
 
 import org.kohsuke.args4j.Option;
 
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
 import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class DescribeCommand extends AbstractCommand {
 
@@ -35,7 +35,7 @@
         String asterixInstanceName = ((DescribeConfig) config).name;
         boolean adminView = ((DescribeConfig) config).admin;
         if (asterixInstanceName != null) {
-            InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
+            AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
                     State.UNUSABLE);
             AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(
                     asterixInstanceName);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
index a2f00b3..68f8532 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
@@ -55,12 +55,26 @@
             case VALIDATE:
                 helpMessage = new ValidateCommand().getUsageDescription();
                 break;
+            case INSTALL:
+                helpMessage = new InstallCommand().getUsageDescription();
+                break;
+            case UNINSTALL:
+                helpMessage = new UninstallCommand().getUsageDescription();
             case ALTER:
                 helpMessage = new AlterCommand().getUsageDescription();
                 break;
             case LOG:
                 helpMessage = new LogCommand().getUsageDescription();
                 break;
+            case STOPNODE:
+                helpMessage = new StopNodeCommand().getUsageDescription();
+                break;
+            case STARTNODE:
+                helpMessage = new StartNodeCommand().getUsageDescription();
+                break;
+            case VERSION:
+                helpMessage = new VersionCommand().getUsageDescription();
+                break;
             default:
                 helpMessage = "Unknown command " + command;
         }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
index 9e67bf5..288c882 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
@@ -27,9 +27,14 @@
         ALTER,
         VALIDATE,
         CONFIGURE,
+        INSTALL,
+        UNINSTALL,
         LOG,
         SHUTDOWN,
-        HELP
+        HELP,
+        STOPNODE,
+        STARTNODE,
+        VERSION
     }
 
     public void execute(String args[]) throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java
new file mode 100644
index 0000000..59a69bf
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2012 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.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class InstallCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig(true);
+        InstallConfig installConfig = ((InstallConfig) config);
+        String instanceName = installConfig.name;
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
+        PatternCreator pc = PatternCreator.INSTANCE;
+        Patterns patterns = pc.getLibraryInstallPattern(instance, installConfig.dataverseName,
+                installConfig.libraryName, installConfig.libraryPath);
+        AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
+        LOGGER.info("Installed library " + installConfig.libraryName);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new InstallConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "Installs a library to an asterix instance." + "\n" + "Arguments/Options\n"
+                + "-n  Name of Asterix Instance\n"
+                + "-d  Name of the dataverse under which the library will be installed\n" + "-l  Name of the library\n"
+                + "-p  Path to library zip bundle";
+
+    }
+
+}
+
+class InstallConfig extends CommandConfig {
+
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
+
+    @Option(name = "-d", required = true, usage = "Name of the dataverse under which the library will be installed")
+    public String dataverseName;
+
+    @Option(name = "-l", required = true, usage = "Name of the library")
+    public String libraryName;
+
+    @Option(name = "-p", required = true, usage = "Path to library zip bundle")
+    public String libraryPath;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
index 11b4aa7..bf30a48 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
@@ -21,14 +21,15 @@
 import org.apache.commons.io.FileUtils;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
 import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
 
 public class LogCommand extends AbstractCommand {
 
@@ -36,12 +37,15 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((LogConfig) config).name;
-        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE,
-                State.UNUSABLE, State.ACTIVE);
-        PatternCreator pc = new PatternCreator();
-        EventrixClient client = InstallerUtil.getEventrixClient(instance.getCluster());
-        String outputDir = ((LogConfig) config).outputDir == null ? InstallerDriver.getManagixHome() + File.separator + "logdump"
-                : ((LogConfig) config).outputDir;
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE, State.UNUSABLE, State.ACTIVE);
+        PatternCreator pc = PatternCreator.INSTANCE;
+
+        AsterixEventServiceClient eventrixClient = AsterixEventService.getAsterixEventServiceClient(
+                instance.getCluster(), true, false);
+
+        String outputDir = ((LogConfig) config).outputDir == null ? InstallerDriver.getManagixHome() + File.separator
+                + "logdump" : ((LogConfig) config).outputDir;
         File f = new File(outputDir);
         String outputDirPath = f.getAbsolutePath();
         if (!f.exists()) {
@@ -50,12 +54,13 @@
                 throw new InstallerException("Unable to create output directory:" + outputDirPath);
             }
         }
-        Patterns transferLogPattern = pc.getGenerateLogPattern(asterixInstanceName, instance.getCluster(), outputDirPath);
-        client.submit(transferLogPattern);
+        Patterns transferLogPattern = pc.getGenerateLogPattern(asterixInstanceName, instance.getCluster(),
+                outputDirPath);
+        eventrixClient.submit(transferLogPattern);
         File outputDirFile = new File(outputDirPath);
         final String destFileName = "log_" + new Date().toString().replace(' ', '_') + ".zip";
         File destFile = new File(outputDirFile, destFileName);
-        InstallerUtil.zipDir(outputDirFile, destFile);
+        AsterixEventServiceUtil.zipDir(outputDirFile, destFile);
 
         String[] filesToDelete = outputDirFile.list(new FilenameFilter() {
             @Override
@@ -65,7 +70,7 @@
 
         });
         for (String fileS : filesToDelete) {
-             f = new File(outputDirFile, fileS);
+            f = new File(outputDirFile, fileS);
             if (f.isDirectory()) {
                 FileUtils.deleteDirectory(f);
             } else {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
index 6627ac2..5b76099 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
@@ -18,13 +18,14 @@
 
 import org.kohsuke.args4j.Option;
 
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.BackupInfo;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
 
 public class RestoreCommand extends AbstractCommand {
 
@@ -32,7 +33,8 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((RestoreConfig) config).name;
-        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE);
         int backupId = ((RestoreConfig) config).backupId;
         List<BackupInfo> backupInfoList = instance.getBackupInfo();
         if (backupInfoList.size() <= backupId || backupId < 0) {
@@ -40,9 +42,8 @@
         }
 
         BackupInfo backupInfo = backupInfoList.get(backupId);
-        PatternCreator pc = new PatternCreator();
-        Patterns patterns = pc.getRestoreAsterixPattern(instance, backupInfo);
-        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        Patterns patterns = PatternCreator.INSTANCE.getRestoreAsterixPattern(instance, backupInfo);
+        AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
         LOGGER.info("Asterix instance: " + asterixInstanceName + " has been restored from backup");
     }
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
index 0c44bd8..228cb1c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
@@ -14,9 +14,10 @@
  */
 package edu.uci.ics.asterix.installer.command;
 
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class ShutdownCommand extends AbstractCommand {
 
@@ -24,7 +25,7 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(false);
         ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
-        lookupService.stopService(InstallerDriver.getConfiguration());
+        lookupService.stopService(AsterixEventService.getConfiguration());
     }
 
     @Override
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
index 3fc7e96..63ec6fc 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
@@ -18,16 +18,17 @@
 
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class StartCommand extends AbstractCommand {
 
@@ -35,18 +36,18 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((StartConfig) config).name;
-        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
-        InstallerUtil.createAsterixZip(instance);
-        PatternCreator pc = new PatternCreator();
-        EventrixClient client = InstallerUtil.getEventrixClient(instance.getCluster());
-        Patterns asterixBinaryTransferPattern = pc.getAsterixBinaryTransferPattern(asterixInstanceName,
-                instance.getCluster());
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE);
+        AsterixEventServiceUtil.createAsterixZip(instance);
+        AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(instance.getCluster());
+        Patterns asterixBinaryTransferPattern = PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
+                asterixInstanceName, instance.getCluster());
         client.submit(asterixBinaryTransferPattern);
-        InstallerUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
-        Patterns patterns = pc.getStartAsterixPattern(asterixInstanceName, instance.getCluster());
+        AsterixEventServiceUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
+        Patterns patterns = PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, instance.getCluster());
         client.submit(patterns);
-        InstallerUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator + InstallerDriver.ASTERIX_DIR
-                + File.separator + asterixInstanceName);
+        AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator
+                + InstallerDriver.ASTERIX_DIR + File.separator + asterixInstanceName);
         AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
         VerificationUtil.updateInstanceWithRuntimeDescription(instance, runtimeState, true);
         LOGGER.info(instance.getDescription(false));
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java
new file mode 100644
index 0000000..7d0a6ee
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java
@@ -0,0 +1,101 @@
+/*
+ * 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.installer.command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+import edu.uci.ics.asterix.installer.error.InstallerException;
+
+public class StartNodeCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig(true);
+        String asterixInstanceName = ((StartNodeConfig) config).name;
+        AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.INACTIVE, State.ACTIVE, State.UNUSABLE);
+
+        Cluster cluster = instance.getCluster();
+        List<Pattern> pl = new ArrayList<Pattern>();
+        AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
+        String[] nodesToBeAdded = ((StartNodeConfig) config).nodes.split(",");
+        List<String> aliveNodes = new ArrayList<String>();
+        for (ProcessInfo p : runtimeState.getProcesses()) {
+            aliveNodes.add(p.getNodeId());
+        }
+        List<Node> clusterNodes = cluster.getNode();
+        for (String n : nodesToBeAdded) {
+            if (aliveNodes.contains(n)) {
+                throw new InstallerException("Node: " + n + " is already alive");
+            }
+            for (Node node : clusterNodes) {
+                if (n.equals(node.getId())) {
+                    String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+                    Pattern createNC = PatternCreator.INSTANCE.createNCStartPattern(cluster.getMasterNode()
+                            .getClusterIp(), node.getId(), asterixInstanceName + "_" + node.getId(), iodevices);
+                    pl.add(createNC);
+                    break;
+                }
+            }
+        }
+        Patterns patterns = new Patterns(pl);
+        AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(cluster);
+        client.submit(patterns);
+        runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
+        VerificationUtil.updateInstanceWithRuntimeDescription(instance, runtimeState, true);
+        LOGGER.info(instance.getDescription(false));
+        ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new StartNodeConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "\nStarts a set of nodes for an ASTERIX instance." + "\n\nAvailable arguments/options"
+                + "\n-n name of the ASTERIX instance. " + "\n-nodes"
+                + "Comma separated list of nodes that need to be started";
+    }
+}
+
+class StartNodeConfig extends CommandConfig {
+
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
+
+    @Option(name = "-nodes", required = true, usage = "Comma separated list of nodes that need to be started")
+    public String nodes;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
index c00fa86..820948c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
@@ -20,16 +20,17 @@
 
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class StopCommand extends AbstractCommand {
 
@@ -37,18 +38,19 @@
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig(true);
         String asterixInstanceName = ((StopConfig) config).name;
-        AsterixInstance asterixInstance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName,
+        AsterixInstance asterixInstance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
                 State.ACTIVE, State.UNUSABLE);
-        PatternCreator pc = new PatternCreator();
-        EventrixClient client = InstallerUtil.getEventrixClient(asterixInstance.getCluster());
+        AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(asterixInstance.getCluster());
 
         List<Pattern> ncKillPatterns = new ArrayList<Pattern>();
         for (Node node : asterixInstance.getCluster().getNode()) {
-            ncKillPatterns.add(pc.createNCStopPattern(node.getId(), asterixInstanceName + "_" + node.getId()));
+            ncKillPatterns.add(PatternCreator.INSTANCE.createNCStopPattern(node.getId(), asterixInstanceName + "_"
+                    + node.getId()));
         }
 
         List<Pattern> ccKillPatterns = new ArrayList<Pattern>();
-        ccKillPatterns.add(pc.createCCStopPattern(asterixInstance.getCluster().getMasterNode().getId()));
+        ccKillPatterns.add(PatternCreator.INSTANCE.createCCStopPattern(asterixInstance.getCluster().getMasterNode()
+                .getId()));
 
         try {
             client.submit(new Patterns(ncKillPatterns));
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java
new file mode 100644
index 0000000..c7cc6de
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java
@@ -0,0 +1,108 @@
+/*
+ * 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.installer.command;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+import edu.uci.ics.asterix.installer.error.InstallerException;
+
+public class StopNodeCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig(true);
+        String asterixInstanceName = ((StopNodeConfig) config).name;
+        AsterixInstance asterixInstance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+                State.ACTIVE, State.UNUSABLE);
+
+        AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(asterixInstance
+                .getCluster());
+
+        String[] nodesToStop = ((StopNodeConfig) config).nodeList.split(",");
+        AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(asterixInstance);
+        List<String> aliveNodes = new ArrayList<String>();
+        for (ProcessInfo p : runtimeState.getProcesses()) {
+            aliveNodes.add(p.getNodeId());
+        }
+
+        List<String> validNodeIds = new ArrayList<String>();
+        for (Node node : asterixInstance.getCluster().getNode()) {
+            validNodeIds.add(node.getId());
+        }
+        List<Pattern> ncKillPatterns = new ArrayList<Pattern>();
+        for (String nodeId : nodesToStop) {
+            if (!nodeId.contains(nodeId)) {
+                throw new InstallerException("Invalid nodeId: " + nodeId);
+            }
+            if (!aliveNodes.contains(nodeId)) {
+                throw new InstallerException("Node: " + nodeId + " is not alive");
+            }
+            ncKillPatterns.add(PatternCreator.INSTANCE.createNCStopPattern(nodeId, asterixInstanceName + "_" + nodeId));
+        }
+
+        try {
+            client.submit(new Patterns(ncKillPatterns));
+        } catch (Exception e) {
+            // processes are already dead
+            LOGGER.debug("Attempt to kill non-existing processess");
+        }
+
+        asterixInstance.setStateChangeTimestamp(new Date());
+        ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(asterixInstance);
+        LOGGER.info("Stopped nodes " + ((StopNodeConfig) config).nodeList + " serving Asterix instance: "
+                + asterixInstanceName);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new StopNodeConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "\nStops a specified set of ASTERIX nodes." + "\n\nAvailable arguments/options"
+                + "\n-n name of the ASTERIX instance. "
+                + "\n-nodes Comma separated list of nodes that need to be stopped. ";
+
+    }
+}
+
+class StopNodeConfig extends CommandConfig {
+
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
+
+    @Option(name = "-nodes", required = true, usage = "Comma separated list of nodes that need to be stopped")
+    public String nodeList;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java
new file mode 100644
index 0000000..39872e7
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2012 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.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class UninstallCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig(true);
+        UninstallConfig uninstallConfig = ((UninstallConfig) config);
+        String instanceName = uninstallConfig.name;
+        AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
+        ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
+        AsterixInstance instance = lookupService.getAsterixInstance(instanceName);
+        PatternCreator pc = PatternCreator.INSTANCE;
+        Patterns patterns = pc.getLibraryUninstallPattern(instance, uninstallConfig.dataverseName,
+                uninstallConfig.libraryName);
+        AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(instance.getCluster());
+        client.submit(patterns);
+        LOGGER.info("Uninstalled library " + uninstallConfig.libraryName);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new UninstallConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "Uninstalls a library from an asterix instance." + "\n" + "Arguments/Options\n"
+                + "-n  Name of Asterix Instance\n"
+                + "-d  Name of the dataverse under which the library will be installed\n" + "-l  Name of the library\n"
+                + "-l  Name of the library";
+    }
+
+}
+
+class UninstallConfig extends CommandConfig {
+
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
+
+    @Option(name = "-d", required = true, usage = "Name of the dataverse under which the library will be installed")
+    public String dataverseName;
+
+    @Option(name = "-l", required = true, usage = "Name of the library")
+    public String libraryName;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
index 5b63166..5035028 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
@@ -29,8 +29,8 @@
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.MasterNode;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 import edu.uci.ics.asterix.installer.schema.conf.Zookeeper;
 
@@ -74,17 +74,10 @@
 
     public boolean validateEnvironment() throws Exception {
         boolean valid = true;
-        String managixHome = System.getenv(InstallerDriver.ENV_MANAGIX_HOME);
-        if (managixHome == null) {
+        File home = new File(InstallerDriver.getManagixHome());
+        if (!home.exists()) {
             valid = false;
-            LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + " not set " + ERROR);
-        } else {
-            File home = new File(managixHome);
-            if (!home.exists()) {
-                valid = false;
-                LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + ": " + home.getAbsolutePath() + " does not exist!"
-                        + ERROR);
-            }
+            LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + ": " + home.getAbsolutePath() + " does not exist!" + ERROR);
         }
         return valid;
 
@@ -109,7 +102,7 @@
 
             MasterNode masterNode = cluster.getMasterNode();
             Node master = new Node(masterNode.getId(), masterNode.getClusterIp(), masterNode.getJavaHome(),
-                    masterNode.getLogDir(), null, null, null);
+                    masterNode.getLogDir(), null, null, null, null);
             ipAddresses.add(masterNode.getClusterIp());
 
             valid = valid & validateNodeConfiguration(master, cluster);
@@ -143,7 +136,7 @@
                 + File.separator + "scripts" + File.separator + "validate_ssh.sh";
         List<String> args = ipAddresses;
         args.add(0, username);
-        String output = InstallerUtil.executeLocalScript(script, args);
+        String output = AsterixEventServiceUtil.executeLocalScript(script, args);
         ipAddresses.remove(0);
         for (String line : output.split("\n")) {
             ipAddresses.remove(line);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java
new file mode 100644
index 0000000..041839d
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java
@@ -0,0 +1,48 @@
+/*
+ * 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.installer.command;
+
+import java.io.File;
+
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class VersionCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig(false);
+        String asterixZipName = AsterixEventService.getAsterixZip().substring(
+                AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+        String asterixVersion = asterixZipName.substring("asterix-server-".length(),
+                asterixZipName.indexOf("-binary-assembly"));
+        LOGGER.info("Asterix/Managix version " + asterixVersion);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new VersionConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "Provides version of Managix/Asterix";
+    }
+
+}
+
+class VersionConfig extends CommandConfig {
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
index 521adc6..53ec136 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
@@ -15,52 +15,38 @@
 package edu.uci.ics.asterix.installer.driver;
 
 import java.io.File;
-import java.io.FileFilter;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.Unmarshaller;
 
-import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
 import edu.uci.ics.asterix.installer.command.CommandHandler;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class InstallerDriver {
 
-    public static final String MANAGIX_INTERNAL_DIR = ".installer";
-    public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix";
-    public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix"
-            + File.separator + "scripts";
-    public static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator + File.separator
-            + "asterix-configuration.xml";
-    public static final String ASTERIX_DIR = "asterix";
-    public static final String EVENTS_DIR = "events";
-
     private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class.getName());
+
+    public static final String MANAGIX_INTERNAL_DIR = ".installer";
     public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
     public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+    public static final String ASTERIX_DIR = "asterix";
 
-    private static Configuration conf;
     private static String managixHome;
-    private static String asterixZip;
-
-    public static String getAsterixZip() {
-        return asterixZip;
-    }
-
-    public static Configuration getConfiguration() {
-        return conf;
-    }
 
     public static void initConfig(boolean ensureLookupServiceIsRunning) throws Exception {
         File configFile = new File(managixHome + File.separator + MANAGIX_CONF_XML);
         JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
         Unmarshaller unmarshaller = configCtx.createUnmarshaller();
-        conf = (Configuration) unmarshaller.unmarshal(configFile);
-        asterixZip = initBinary("asterix-server");
+        Configuration conf = (Configuration) unmarshaller.unmarshal(configFile);
+        String asterixDir = managixHome + File.separator + ASTERIX_DIR;
+        String eventHome = managixHome + File.separator + MANAGIX_INTERNAL_DIR;
+        AsterixEventService.initialize(conf, asterixDir, eventHome);
 
         ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
         if (ensureLookupServiceIsRunning && !lookupService.isRunning(conf)) {
@@ -68,28 +54,6 @@
         }
     }
 
-    private static String initBinary(final String fileNamePattern) {
-        String asterixDir = InstallerDriver.getAsterixDir();
-        File file = new File(asterixDir);
-        File[] zipFiles = file.listFiles(new FileFilter() {
-            public boolean accept(File arg0) {
-                return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
-            }
-        });
-        if (zipFiles.length == 0) {
-            String msg = " Binary not found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
-        if (zipFiles.length > 1) {
-            String msg = " Multiple binaries found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
-
-        return zipFiles[0].getAbsolutePath();
-    }
-
     public static String getManagixHome() {
         return managixHome;
     }
@@ -98,10 +62,6 @@
         InstallerDriver.managixHome = managixHome;
     }
 
-    public static String getAsterixDir() {
-        return managixHome + File.separator + ASTERIX_DIR;
-    }
-
     public static void main(String args[]) {
         try {
             if (args.length != 0) {
@@ -134,12 +94,18 @@
         buffer.append("alter    " + ":" + " Alter the instance's configuration settings" + "\n");
         buffer.append("describe " + ":" + " Describes an existing asterix instance" + "\n");
         buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
-        buffer.append("configure" + ":" + " Configure the Asterix installer" + "\n");
+        buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
+                + "\n");
+        buffer.append("install  " + ":" + " Installs a library to an asterix instance" + "\n");
+        buffer.append("uninstall" + ":" + " Uninstalls a library from an asterix instance" + "\n");
         buffer.append("log      " + ":"
                 + " Produce a tar archive contianing log files from the master and worker nodes" + "\n");
         buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
         buffer.append("help     " + ":" + " Provides usage description of a command" + "\n");
+        buffer.append("version  " + ":" + " Provides version of Asterix/Managix" + "\n");
+
         buffer.append("\nTo get more information about a command, use managix help -cmd <command>");
         LOGGER.info(buffer.toString());
     }
+
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index 2e04bbf..2569363 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -14,58 +14,23 @@
  */
 package edu.uci.ics.asterix.installer.driver;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.File;
-import java.io.FileFilter;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FileWriter;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-import java.util.jar.JarOutputStream;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-import java.util.zip.ZipOutputStream;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
-import org.apache.commons.io.IOUtils;
-
 import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
-import edu.uci.ics.asterix.common.configuration.Coredump;
-import edu.uci.ics.asterix.common.configuration.Store;
-import edu.uci.ics.asterix.common.configuration.TransactionLogDir;
-import edu.uci.ics.asterix.event.driver.EventDriver;
-import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
-import edu.uci.ics.asterix.event.schema.cluster.Env;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.event.schema.cluster.Property;
-import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.error.OutputHandler;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 
 public class InstallerUtil {
 
+    private static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator
+            + "asterix-configuration.xml";
+
     public static final String TXN_LOG_DIR = "txnLogs";
     public static final String TXN_LOG_DIR_KEY_SUFFIX = "txnLogDir";
     public static final String ASTERIX_CONFIGURATION_FILE = "asterix-configuration.xml";
@@ -75,133 +40,6 @@
     public static final int HTTP_PORT_DEFAULT = 8888;
     public static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
 
-    public static AsterixInstance createAsterixInstance(String asterixInstanceName, Cluster cluster,
-            AsterixConfiguration asterixConfiguration) throws FileNotFoundException, IOException {
-        Node metadataNode = getMetadataNode(cluster);
-        String asterixZipName = InstallerDriver.getAsterixZip().substring(
-                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-        String asterixVersion = asterixZipName.substring("asterix-server-".length(),
-                asterixZipName.indexOf("-binary-assembly"));
-        AsterixInstance instance = new AsterixInstance(asterixInstanceName, cluster, asterixConfiguration,
-                metadataNode.getId(), asterixVersion);
-        return instance;
-    }
-
-    public static void createAsterixZip(AsterixInstance asterixInstance) throws IOException, InterruptedException,
-            JAXBException, InstallerException {
-
-        String modifiedZipPath = injectAsterixPropertyFile(InstallerDriver.getAsterixZip(), asterixInstance);
-        injectAsterixLogPropertyFile(modifiedZipPath, asterixInstance);
-    }
-
-    public static void createClusterProperties(Cluster cluster, AsterixConfiguration asterixConfiguration) {
-        List<Property> clusterProperties = null;
-        if (cluster.getEnv() != null && cluster.getEnv().getProperty() != null) {
-            clusterProperties = cluster.getEnv().getProperty();
-            clusterProperties.clear();
-        } else {
-            clusterProperties = new ArrayList<Property>();
-        }
-        for (edu.uci.ics.asterix.common.configuration.Property property : asterixConfiguration.getProperty()) {
-            if (property.getName().equalsIgnoreCase(EventUtil.CC_JAVA_OPTS)) {
-                clusterProperties.add(new Property(EventUtil.CC_JAVA_OPTS, property.getValue()));
-            } else if (property.getName().equalsIgnoreCase(EventUtil.NC_JAVA_OPTS)) {
-                clusterProperties.add(new Property(EventUtil.NC_JAVA_OPTS, property.getValue()));
-            }
-        }
-        clusterProperties.add(new Property("ASTERIX_HOME", cluster.getWorkingDir().getDir() + File.separator
-                + "asterix"));
-        clusterProperties.add(new Property("LOG_DIR", cluster.getLogDir()));
-        clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
-        clusterProperties.add(new Property("WORKING_DIR", cluster.getWorkingDir().getDir()));
-        clusterProperties.add(new Property("CLIENT_NET_IP", cluster.getMasterNode().getClientIp()));
-        clusterProperties.add(new Property("CLUSTER_NET_IP", cluster.getMasterNode().getClusterIp()));
-
-        int clusterNetPort = cluster.getMasterNode().getClusterPort() != null ? cluster.getMasterNode()
-                .getClusterPort().intValue() : CLUSTER_NET_PORT_DEFAULT;
-        int clientNetPort = cluster.getMasterNode().getClientPort() != null ? cluster.getMasterNode().getClientPort()
-                .intValue() : CLIENT_NET_PORT_DEFAULT;
-        int httpPort = cluster.getMasterNode().getHttpPort() != null ? cluster.getMasterNode().getHttpPort().intValue()
-                : HTTP_PORT_DEFAULT;
-
-        clusterProperties.add(new Property("CLIENT_NET_PORT", "" + clientNetPort));
-        clusterProperties.add(new Property("CLUSTER_NET_PORT", "" + clusterNetPort));
-        clusterProperties.add(new Property("HTTP_PORT", "" + httpPort));
-
-        cluster.setEnv(new Env(clusterProperties));
-    }
-
-    private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance)
-            throws IOException, JAXBException {
-        writeAsterixConfigurationFile(asterixInstance);
-        String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
-        unzip(origZipFile, asterixInstanceDir);
-        File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
-                + asterixInstance.getAsterixVersion() + ".jar");
-        File replacementFile = new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE);
-        replaceInJar(sourceJar, ASTERIX_CONFIGURATION_FILE, replacementFile);
-        new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE).delete();
-        String asterixZipName = InstallerDriver.getAsterixZip().substring(
-                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
-        return asterixInstanceDir + File.separator + asterixZipName;
-    }
-
-    private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
-            throws IOException, InstallerException {
-        String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
-        unzip(origZipFile, asterixInstanceDir);
-        File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
-                + asterixInstance.getAsterixVersion() + ".jar");
-        Properties txnLogProperties = new Properties();
-        URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
-        InputStream in = urlClassLoader.getResourceAsStream(TXN_LOG_CONFIGURATION_FILE);
-        if (in != null) {
-            txnLogProperties.load(in);
-        }
-
-        writeAsterixLogConfigurationFile(asterixInstance, txnLogProperties);
-
-        File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
-                + asterixInstance.getAsterixVersion() + ".jar");
-        File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
-        replaceInJar(sourceJar2, TXN_LOG_CONFIGURATION_FILE, replacementFile);
-
-        new File(asterixInstanceDir + File.separator + "log.properties").delete();
-        String asterixZipName = InstallerDriver.getAsterixZip().substring(
-                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
-        return asterixInstanceDir + File.separator + asterixZipName;
-    }
-
-    public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
-            String libraryName, String libraryPath) throws IOException {
-        File instanceDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName());
-        if (!instanceDir.exists()) {
-            instanceDir.mkdirs();
-        }
-        String asterixZipName = InstallerDriver.getAsterixZip().substring(
-                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-
-        String sourceZip = instanceDir.getAbsolutePath() + File.separator + asterixZipName;
-        unzip(sourceZip, instanceDir.getAbsolutePath());
-        File libraryPathInZip = new File(instanceDir.getAbsolutePath() + File.separator + "external" + File.separator
-                + "library" + dataverseName + File.separator + "to-add" + File.separator + libraryName);
-        libraryPathInZip.mkdirs();
-        Runtime.getRuntime().exec("cp" + " " + libraryPath + " " + libraryPathInZip.getAbsolutePath());
-        Runtime.getRuntime().exec("rm " + sourceZip);
-        String destZip = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName() + File.separator
-                + asterixZipName;
-        zipDir(instanceDir, new File(destZip));
-        Runtime.getRuntime().exec("mv" + " " + destZip + " " + sourceZip);
-    }
-
-    private static Node getMetadataNode(Cluster cluster) {
-        Random random = new Random();
-        int nNodes = cluster.getNode().size();
-        return cluster.getNode().get(random.nextInt(nNodes));
-    }
-
     public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
         String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
         String[] storeDirs = null;
@@ -224,83 +62,10 @@
         return nodeDataStore.toString();
     }
 
-    private static void writeAsterixConfigurationFile(AsterixInstance asterixInstance) throws IOException,
-            JAXBException {
-        String asterixInstanceName = asterixInstance.getName();
-        Cluster cluster = asterixInstance.getCluster();
-        String metadataNodeId = asterixInstance.getMetadataNodeId();
-
-        AsterixConfiguration configuration = asterixInstance.getAsterixConfiguration();
-        configuration.setMetadataNode(asterixInstanceName + "_" + metadataNodeId);
-
-        String storeDir = null;
-        List<Store> stores = new ArrayList<Store>();
-        for (Node node : cluster.getNode()) {
-            storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
-            stores.add(new Store(asterixInstanceName + "_" + node.getId(), storeDir));
-        }
-        configuration.setStore(stores);
-
-        List<Coredump> coredump = new ArrayList<Coredump>();
-        String coredumpDir = null;
-        List<TransactionLogDir> txnLogDirs = new ArrayList<TransactionLogDir>();
-        String txnLogDir = null;
-        for (Node node : cluster.getNode()) {
-            coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
-            coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator
-                    + asterixInstanceName + "_" + node.getId()));
-
-            txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
-            txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
-        }
-        configuration.setCoredump(coredump);
-        configuration.setTransactionLogDir(txnLogDirs);
-
-        File asterixConfDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName);
-        asterixConfDir.mkdirs();
-
-        JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
-        Marshaller marshaller = ctx.createMarshaller();
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(configuration, new FileOutputStream(asterixConfDir + File.separator
-                + ASTERIX_CONFIGURATION_FILE));
-    }
-
-    private static void writeAsterixLogConfigurationFile(AsterixInstance asterixInstance, Properties logProperties)
-            throws IOException, InstallerException {
-        String asterixInstanceName = asterixInstance.getName();
-        Cluster cluster = asterixInstance.getCluster();
-        StringBuffer conf = new StringBuffer();
-        for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
-            conf.append(p.getKey() + "=" + p.getValue() + "\n");
-        }
-
-        for (Node node : cluster.getNode()) {
-            String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
-            if (txnLogDir == null) {
-                throw new InstallerException("Transaction log directory (txn_log_dir) not configured for node: "
-                        + node.getId());
-            }
-            conf.append(asterixInstanceName + "_" + node.getId() + "." + TXN_LOG_DIR_KEY_SUFFIX + "=" + txnLogDir
-                    + "\n");
-        }
-        List<edu.uci.ics.asterix.common.configuration.Property> properties = asterixInstance.getAsterixConfiguration()
-                .getProperty();
-        for (edu.uci.ics.asterix.common.configuration.Property p : properties) {
-            if (p.getName().trim().toLowerCase().contains("log")) {
-                conf.append(p.getValue() + "=" + p.getValue());
-            }
-        }
-        dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
-                + "log.properties", conf.toString());
-
-    }
-
     public static AsterixConfiguration getAsterixConfiguration(String asterixConf) throws FileNotFoundException,
             IOException, JAXBException {
         if (asterixConf == null) {
-            asterixConf = InstallerDriver.getManagixHome() + File.separator
-                    + InstallerDriver.DEFAULT_ASTERIX_CONFIGURATION_PATH;
+            asterixConf = InstallerDriver.getManagixHome() + File.separator + DEFAULT_ASTERIX_CONFIGURATION_PATH;
         }
         File file = new File(asterixConf);
         JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
@@ -309,199 +74,4 @@
         return asterixConfiguration;
     }
 
-    public static void unzip(String sourceFile, String destDir) throws IOException {
-        BufferedOutputStream dest = null;
-        FileInputStream fis = new FileInputStream(sourceFile);
-        ZipInputStream zis = new ZipInputStream(new BufferedInputStream(fis));
-        ZipEntry entry = null;
-
-        int BUFFER_SIZE = 4096;
-        while ((entry = zis.getNextEntry()) != null) {
-            String dst = destDir + File.separator + entry.getName();
-            if (entry.isDirectory()) {
-                createDir(destDir, entry);
-                continue;
-            }
-            int count;
-            byte data[] = new byte[BUFFER_SIZE];
-
-            // write the file to the disk
-            FileOutputStream fos = new FileOutputStream(dst);
-            dest = new BufferedOutputStream(fos, BUFFER_SIZE);
-            while ((count = zis.read(data, 0, BUFFER_SIZE)) != -1) {
-                dest.write(data, 0, count);
-            }
-            // close the output streams
-            dest.flush();
-            dest.close();
-        }
-
-        zis.close();
-    }
-
-    public static void zipDir(File sourceDir, File destFile) throws IOException {
-        FileOutputStream fos = new FileOutputStream(destFile);
-        ZipOutputStream zos = new ZipOutputStream(fos);
-        zipDir(sourceDir, destFile, zos);
-        zos.close();
-    }
-
-    private static void zipDir(File sourceDir, final File destFile, ZipOutputStream zos) throws IOException {
-        File[] dirList = sourceDir.listFiles(new FileFilter() {
-            public boolean accept(File f) {
-                return !f.getName().endsWith(destFile.getName());
-            }
-        });
-        for (int i = 0; i < dirList.length; i++) {
-            File f = dirList[i];
-            if (f.isDirectory()) {
-                zipDir(f, destFile, zos);
-            } else {
-                int bytesIn = 0;
-                byte[] readBuffer = new byte[2156];
-                FileInputStream fis = new FileInputStream(f);
-                ZipEntry entry = new ZipEntry(sourceDir.getName() + File.separator + f.getName());
-                zos.putNextEntry(entry);
-                while ((bytesIn = fis.read(readBuffer)) != -1) {
-                    zos.write(readBuffer, 0, bytesIn);
-                }
-                fis.close();
-            }
-        }
-    }
-
-    private static void replaceInJar(File sourceJar, String origFile, File replacementFile) throws IOException {
-        File destJar = new File(sourceJar.getAbsolutePath() + ".modified");
-        InputStream jarIs = null;
-        FileInputStream fis = new FileInputStream(replacementFile);
-        JarFile sourceJarFile = new JarFile(sourceJar);
-        Enumeration<JarEntry> entries = sourceJarFile.entries();
-        JarOutputStream jos = new JarOutputStream(new FileOutputStream(destJar));
-        byte[] buffer = new byte[2048];
-        int read;
-        while (entries.hasMoreElements()) {
-            JarEntry entry = (JarEntry) entries.nextElement();
-            String name = entry.getName();
-            if (name.equals(origFile)) {
-                continue;
-            }
-            jarIs = sourceJarFile.getInputStream(entry);
-            jos.putNextEntry(entry);
-            while ((read = jarIs.read(buffer)) != -1) {
-                jos.write(buffer, 0, read);
-            }
-        }
-        JarEntry entry = new JarEntry(origFile);
-        jos.putNextEntry(entry);
-        while ((read = fis.read(buffer)) != -1) {
-            jos.write(buffer, 0, read);
-        }
-        fis.close();
-        jos.close();
-        jarIs.close();
-        sourceJar.delete();
-        destJar.renameTo(sourceJar);
-        sourceJar.setExecutable(true);
-    }
-
-    public static void dumpToFile(String dest, String content) throws IOException {
-        FileWriter writer = new FileWriter(dest);
-        writer.write(content);
-        writer.close();
-    }
-
-    private static void createDir(String destDirectory, ZipEntry entry) {
-        String name = entry.getName();
-        int index = name.lastIndexOf(File.separator);
-        String dirSequence = name.substring(0, index);
-        File newDirs = new File(destDirectory + File.separator + dirSequence);
-        newDirs.mkdirs();
-    }
-
-    public static AsterixInstance validateAsterixInstanceExists(String name, State... permissibleStates)
-            throws Exception {
-        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
-        if (instance == null) {
-            throw new InstallerException("Asterix instance by name " + name + " does not exist.");
-        }
-        boolean valid = false;
-        for (State state : permissibleStates) {
-            if (state.equals(instance.getState())) {
-                valid = true;
-                break;
-            }
-        }
-        if (!valid) {
-            throw new InstallerException("Asterix instance by the name " + name + " is in " + instance.getState()
-                    + " state ");
-        }
-        return instance;
-    }
-
-    public static void validateAsterixInstanceNotExists(String name) throws Exception {
-        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
-        if (instance != null) {
-            throw new InstallerException("Asterix instance by name " + name + " already exists.");
-        }
-    }
-
-    public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
-        List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
-        List<String> usedIps = new ArrayList<String>();
-        String masterIp = instance.getCluster().getMasterNode().getClusterIp();
-        for (Node node : instance.getCluster().getNode()) {
-            usedIps.add(node.getClusterIp());
-        }
-        usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
-        boolean conflictFound = false;
-        AsterixInstance conflictingInstance = null;
-        for (AsterixInstance existing : existingInstances) {
-            if (!existing.getState().equals(State.INACTIVE)) {
-                conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
-                if (conflictFound) {
-                    conflictingInstance = existing;
-                    break;
-                }
-                for (Node n : existing.getCluster().getNode()) {
-                    if (usedIps.contains(n.getClusterIp())) {
-                        conflictFound = true;
-                        conflictingInstance = existing;
-                        break;
-                    }
-                }
-            }
-        }
-        if (conflictFound) {
-            throw new Exception("Cluster definition conflicts with an existing instance of Asterix: "
-                    + conflictingInstance.getName());
-        }
-    }
-
-    public static void deleteDirectory(String path) throws IOException {
-        Runtime.getRuntime().exec("rm -rf " + path);
-    }
-
-    public static String executeLocalScript(String path, List<String> args) throws Exception {
-        List<String> pargs = new ArrayList<String>();
-        pargs.add("/bin/bash");
-        pargs.add(path);
-        if (args != null) {
-            pargs.addAll(args);
-        }
-        ProcessBuilder pb = new ProcessBuilder(pargs);
-        pb.environment().putAll(EventDriver.getEnvironment());
-        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
-        Process p = pb.start();
-        BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
-        StringWriter writer = new StringWriter();
-        IOUtils.copy(bis, writer, "UTF-8");
-        return writer.toString();
-    }
-
-    public static EventrixClient getEventrixClient(Cluster cluster) throws Exception {
-        return new EventrixClient(
-                InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_EVENT_DIR, cluster, false,
-                OutputHandler.INSTANCE);
-    }
-
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
index 313fa50..0a86196 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
@@ -20,8 +20,8 @@
 
 import edu.uci.ics.asterix.event.management.IOutputHandler;
 import edu.uci.ics.asterix.event.management.OutputAnalysis;
+import edu.uci.ics.asterix.event.model.EventList.EventType;
 import edu.uci.ics.asterix.event.schema.pattern.Event;
-import edu.uci.ics.asterix.installer.model.EventList.EventType;
 
 public class OutputHandler implements IOutputHandler {
 
diff --git a/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop b/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop-config.sh b/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop-config.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/managix b/asterix-installer/src/main/resources/scripts/managix
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/validate_ssh.sh b/asterix-installer/src/main/resources/scripts/validate_ssh.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/verify.sh b/asterix-installer/src/main/resources/scripts/verify.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/start_zk.sh b/asterix-installer/src/main/resources/zookeeper/start_zk.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/stop_zk b/asterix-installer/src/main/resources/zookeeper/stop_zk
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/zk.init b/asterix-installer/src/main/resources/zookeeper/zk.init
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java
new file mode 100644
index 0000000..daa8bb5
--- /dev/null
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java
@@ -0,0 +1,85 @@
+/*
+ * 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.installer.test;
+
+import java.io.File;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.test.aql.TestsUtils;
+import edu.uci.ics.asterix.testframework.context.TestCaseContext;
+
+public class AsterixExternalLibraryIT {
+
+	private static final String LIBRARY_NAME = "testlib";
+	private static final String LIBRARY_DATAVERSE = "externallibtest";
+	private static final String PATH_BASE = "src/test/resources/integrationts/library";
+	private static final String PATH_ACTUAL = "ittest/";
+	private static final String LIBRARY_PATH = "asterix-external-data"
+			+ File.separator + "target" + File.separator
+			+ "testlib-zip-binary-assembly.zip";
+	private static final Logger LOGGER = Logger
+			.getLogger(AsterixExternalLibraryIT.class.getName());
+	private static List<TestCaseContext> testCaseCollection;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		AsterixInstallerIntegrationUtil.init();
+		File asterixInstallerProjectDir = new File(System
+				.getProperty("user.dir"));
+		String asterixExternalLibraryPath = asterixInstallerProjectDir
+				.getParentFile().getAbsolutePath()
+				+ File.separator + LIBRARY_PATH;
+		LOGGER.info("Installing library :" + LIBRARY_NAME + " located at "
+				+ asterixExternalLibraryPath + " in dataverse "
+				+ LIBRARY_DATAVERSE);
+		AsterixInstallerIntegrationUtil.installLibrary(LIBRARY_NAME,
+				LIBRARY_DATAVERSE, asterixExternalLibraryPath);
+		AsterixInstallerIntegrationUtil
+				.transformIntoRequiredState(State.ACTIVE);
+		TestCaseContext.Builder b = new TestCaseContext.Builder();
+		testCaseCollection = b.build(new File(PATH_BASE));
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		AsterixInstallerIntegrationUtil.deinit();
+	}
+
+	@Test
+	public void test() throws Exception {
+		for (TestCaseContext testCaseCtx : testCaseCollection) {
+			TestsUtils.executeTest(PATH_ACTUAL, testCaseCtx);
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+		try {
+			setUp();
+			new AsterixExternalLibraryIT().test();
+		} catch (Exception e) {
+		    e.printStackTrace();
+			LOGGER.info("TEST CASES FAILED");
+		} finally {
+			tearDown();
+		}
+	}
+
+}
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java
new file mode 100644
index 0000000..8a126a2
--- /dev/null
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java
@@ -0,0 +1,82 @@
+/*
+ * 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.installer.test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
+
+import edu.uci.ics.asterix.test.aql.TestsUtils;
+import edu.uci.ics.asterix.testframework.context.TestCaseContext;
+
+public class AsterixFaultToleranceIT {
+
+    private static final String PATH_BASE = "src/test/resources/integrationts/fault-tolerance";
+    private static final String PATH_ACTUAL = "ittest/";
+    private static final Logger LOGGER = Logger.getLogger(AsterixFaultToleranceIT.class.getName());
+    private static List<TestCaseContext> testCaseCollection;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        AsterixInstallerIntegrationUtil.init();
+        TestCaseContext.Builder b = new TestCaseContext.Builder();
+        testCaseCollection = b.build(new File(PATH_BASE));
+        File outdir = new File(PATH_ACTUAL);
+        outdir.mkdirs();
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        AsterixInstallerIntegrationUtil.deinit();
+        File outdir = new File(PATH_ACTUAL);
+        File[] files = outdir.listFiles();
+        if (files == null || files.length == 0) {
+            outdir.delete();
+        }
+    }
+
+    @Parameters
+    public static Collection<Object[]> tests() throws Exception {
+        Collection<Object[]> testArgs = new ArrayList<Object[]>();
+        return testArgs;
+    }
+
+    @Test
+    public void test() throws Exception {
+        for (TestCaseContext testCaseCtx : testCaseCollection) {
+             TestsUtils.executeTest(PATH_ACTUAL, testCaseCtx);
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        try {
+            setUp();
+            new AsterixFaultToleranceIT().test();
+        } catch (Exception e) {
+            e.printStackTrace();
+            LOGGER.info("TEST CASE(S) FAILED");
+        } finally {
+            tearDown();
+        }
+    }
+
+}
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index c7beb28..1096a4d 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -28,191 +28,210 @@
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.command.CommandHandler;
-import edu.uci.ics.asterix.installer.command.ShutdownCommand;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 
 public class AsterixInstallerIntegrationUtil {
 
-    private static String managixHome;
-    private static String clusterConfigurationPath;
-    private static final CommandHandler cmdHandler = new CommandHandler();
-    public static final String ASTERIX_INSTANCE_NAME = "asterix";
-    private static final String CC_IP_ADDRESS = "127.0.0.1";
-    private static final int DEFAULT_HYRACKS_CC_CLIENT_PORT = 1098;
-    private static final int zookeeperClientPort = 2900;
-    private static final int zookeeperTestClientPort = 3945;
+	private static String managixHome;
+	private static String clusterConfigurationPath;
+	private static final CommandHandler cmdHandler = new CommandHandler();
+	public static final String ASTERIX_INSTANCE_NAME = "asterix";
+	private static final String CC_IP_ADDRESS = "127.0.0.1";
+	private static final int DEFAULT_HYRACKS_CC_CLIENT_PORT = 1098;
+	private static final int zookeeperClientPort = 2900;
+	private static final int zookeeperTestClientPort = 3945;
 
-    private static IHyracksClientConnection hcc;
+	private static IHyracksClientConnection hcc;
 
-    private static final Logger LOGGER = Logger.getLogger(AsterixInstallerIntegrationUtil.class.getName());
+	private static final Logger LOGGER = Logger
+			.getLogger(AsterixInstallerIntegrationUtil.class.getName());
 
-    public static void deinit() throws Exception {
-        deleteInstance();
-        stopZookeeper();
-    }
+	public static void deinit() throws Exception {
+		deleteInstance();
+		stopZookeeper();
+	}
 
-    public static void init() throws Exception {
-        File asterixProjectDir = new File(System.getProperty("user.dir"));
-        File installerTargetDir = new File(asterixProjectDir, "target");
-        String managixHomeDirName = installerTargetDir.list(new FilenameFilter() {
-            @Override
-            public boolean accept(File dir, String name) {
-                return new File(dir, name).isDirectory() && name.startsWith("asterix-installer")
-                        && name.endsWith("binary-assembly");
-            }
+	public static void init() throws Exception {
+		File asterixProjectDir = new File(System.getProperty("user.dir"));
+		File installerTargetDir = new File(asterixProjectDir, "target");
+		String managixHomeDirName = installerTargetDir
+				.list(new FilenameFilter() {
+					@Override
+					public boolean accept(File dir, String name) {
+						return new File(dir, name).isDirectory()
+								&& name.startsWith("asterix-installer")
+								&& name.endsWith("binary-assembly");
+					}
 
-        })[0];
-        managixHome = new File(installerTargetDir, managixHomeDirName).getAbsolutePath();
-        System.setProperty("log4j.configuration", managixHome + File.separator + "conf" + File.separator
-                + "log4j.properties");
+				})[0];
+		managixHome = new File(installerTargetDir, managixHomeDirName)
+				.getAbsolutePath();
+		System.setProperty("log4j.configuration", managixHome + File.separator
+				+ "conf" + File.separator + "log4j.properties");
 
-        managixHome = AsterixInstallerIntegrationUtil.getManagixHome();
-        clusterConfigurationPath = managixHome + File.separator + "clusters" + File.separator + "local"
-                + File.separator + "local.xml";
+		managixHome = AsterixInstallerIntegrationUtil.getManagixHome();
+		clusterConfigurationPath = managixHome + File.separator + "clusters"
+				+ File.separator + "local" + File.separator + "local.xml";
 
-        InstallerDriver.setManagixHome(managixHome);
+		InstallerDriver.setManagixHome(managixHome);
 
-        String command = "configure";
-        cmdHandler.processCommand(command.split(" "));
-        command = "validate -c " + clusterConfigurationPath;
-        cmdHandler.processCommand(command.split(" "));
+		String command = "configure";
+		cmdHandler.processCommand(command.split(" "));
+		command = "validate -c " + clusterConfigurationPath;
+		cmdHandler.processCommand(command.split(" "));
 
-        startZookeeper();
-        InstallerDriver.initConfig(true);
-        createInstance();
-        hcc = new HyracksConnection(CC_IP_ADDRESS, DEFAULT_HYRACKS_CC_CLIENT_PORT);
-    }
+		startZookeeper();
+		Thread.sleep(2000);
+		InstallerDriver.initConfig(true);
+		createInstance();
+		hcc = new HyracksConnection(CC_IP_ADDRESS,
+				DEFAULT_HYRACKS_CC_CLIENT_PORT);
+	}
 
-    public static IHyracksClientConnection getHyracksConnection() {
-        return hcc;
-    }
+	public static IHyracksClientConnection getHyracksConnection() {
+		return hcc;
+	}
 
-    private static void startZookeeper() throws Exception {
-        initZookeeperTestConfiguration(zookeeperClientPort);
-        String script = managixHome + File.separator + "bin" + File.separator + "managix";
+	private static void startZookeeper() throws Exception {
+		initZookeeperTestConfiguration(zookeeperClientPort);
+		String script = managixHome + File.separator + "bin" + File.separator
+				+ "managix";
 
-        // shutdown zookeeper if running
-        String command = "shutdown";
-        cmdHandler.processCommand(command.split(" "));
+		// shutdown zookeeper if running
+		String command = "shutdown";
+		cmdHandler.processCommand(command.split(" "));
 
-         Thread.sleep(2000);
+		Thread.sleep(2000);
 
-        // start zookeeper 
-        initZookeeperTestConfiguration(zookeeperTestClientPort);
-        ProcessBuilder pb2 = new ProcessBuilder(script, "describe");
-        Map<String, String> env2 = pb2.environment();
-        env2.put("MANAGIX_HOME", managixHome);
-        pb2.start();
+		// start zookeeper
+		initZookeeperTestConfiguration(zookeeperTestClientPort);
+		ProcessBuilder pb2 = new ProcessBuilder(script, "describe");
+		Map<String, String> env2 = pb2.environment();
+		env2.put("MANAGIX_HOME", managixHome);
+		pb2.start();
 
-        Thread.sleep(2000);
-    }
+	}
 
-    public static void createInstance() throws Exception {
+	public static void createInstance() throws Exception {
 
-        String command = null;
-        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
-                .getAsterixInstance(ASTERIX_INSTANCE_NAME);
-        if (instance != null) {
-            transformIntoRequiredState(State.INACTIVE);
-            command = "delete -n " + ASTERIX_INSTANCE_NAME;
-            cmdHandler.processCommand(command.split(" "));
-        }
+		String command = null;
+		AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(ASTERIX_INSTANCE_NAME);
+		if (instance != null) {
+			transformIntoRequiredState(State.INACTIVE);
+			command = "delete -n " + ASTERIX_INSTANCE_NAME;
+			cmdHandler.processCommand(command.split(" "));
+		}
 
-        command = "create -n " + ASTERIX_INSTANCE_NAME + " " + "-c" + " " + clusterConfigurationPath;
-        cmdHandler.processCommand(command.split(" "));
+		command = "create -n " + ASTERIX_INSTANCE_NAME + " " + "-c" + " "
+				+ clusterConfigurationPath;
+		cmdHandler.processCommand(command.split(" "));
 
-        instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(ASTERIX_INSTANCE_NAME);
-        AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
-        assert (state.getFailedNCs().isEmpty() && state.isCcRunning());
-    }
+		instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(ASTERIX_INSTANCE_NAME);
+		AsterixRuntimeState state = VerificationUtil
+				.getAsterixRuntimeState(instance);
+		assert (state.getFailedNCs().isEmpty() && state.isCcRunning());
+	}
 
-    private static void initZookeeperTestConfiguration(int port) throws JAXBException, FileNotFoundException {
-        String installerConfPath = InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_CONF_XML;
-        JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
-        Unmarshaller unmarshaller = ctx.createUnmarshaller();
-        Configuration configuration = (Configuration) unmarshaller.unmarshal(new File(installerConfPath));
-        configuration.getZookeeper().setClientPort(new BigInteger("" + port));
-        Marshaller marshaller = ctx.createMarshaller();
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(configuration, new FileOutputStream(installerConfPath));
-    }
+	private static void initZookeeperTestConfiguration(int port)
+			throws JAXBException, FileNotFoundException {
+		String installerConfPath = InstallerDriver.getManagixHome()
+				+ File.separator + InstallerDriver.MANAGIX_CONF_XML;
+		JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
+		Unmarshaller unmarshaller = ctx.createUnmarshaller();
+		Configuration configuration = (Configuration) unmarshaller
+				.unmarshal(new File(installerConfPath));
+		configuration.getZookeeper().setClientPort(new BigInteger("" + port));
+		Marshaller marshaller = ctx.createMarshaller();
+		marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+		marshaller.marshal(configuration, new FileOutputStream(
+				installerConfPath));
+	}
 
-    public static void transformIntoRequiredState(AsterixInstance.State state) throws Exception {
-        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
-                .getAsterixInstance(ASTERIX_INSTANCE_NAME);
-        assert (instance != null);
-        if (instance.getState().equals(state)) {
-            return;
-        }
-        if (state.equals(AsterixInstance.State.UNUSABLE)) {
-            throw new IllegalArgumentException("Invalid desired state");
-        }
+	public static void transformIntoRequiredState(AsterixInstance.State state)
+			throws Exception {
+		AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(ASTERIX_INSTANCE_NAME);
+		assert (instance != null);
+		if (instance.getState().equals(state)) {
+			return;
+		}
+		if (state.equals(AsterixInstance.State.UNUSABLE)) {
+			throw new IllegalArgumentException("Invalid desired state");
+		}
 
-        String command = null;
-        switch (instance.getState()) {
-            case ACTIVE:
-                command = "stop -n " + ASTERIX_INSTANCE_NAME;
-                break;
-            case INACTIVE:
-                command = "start -n " + ASTERIX_INSTANCE_NAME;
-                break;
-        }
-        cmdHandler.processCommand(command.split(" "));
-    }
+		String command = null;
+		switch (instance.getState()) {
+		case ACTIVE:
+			command = "stop -n " + ASTERIX_INSTANCE_NAME;
+			break;
+		case INACTIVE:
+			command = "start -n " + ASTERIX_INSTANCE_NAME;
+			break;
+		}
+		cmdHandler.processCommand(command.split(" "));
+	}
 
-    private static void stopZookeeper() throws IOException, JAXBException {
-        String script = managixHome + File.separator + "bin" + File.separator + "managix";
-        // shutdown zookeeper if running
-        ProcessBuilder pb = new ProcessBuilder(script, "shutdown");
-        Map<String, String> env = pb.environment();
-        env.put("MANAGIX_HOME", managixHome);
-        pb.start();
-    }
+	private static void stopZookeeper() throws IOException, JAXBException {
+		String script = managixHome + File.separator + "bin" + File.separator
+				+ "managix";
+		// shutdown zookeeper if running
+		ProcessBuilder pb = new ProcessBuilder(script, "shutdown");
+		Map<String, String> env = pb.environment();
+		env.put("MANAGIX_HOME", managixHome);
+		pb.start();
+	}
 
-    private static void deleteInstance() throws Exception {
-        String command = null;
-        AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
-                .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+	private static void deleteInstance() throws Exception {
+		String command = null;
+		AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(ASTERIX_INSTANCE_NAME);
 
-        if (instance == null) {
-            return;
-        } else {
-            transformIntoRequiredState(State.INACTIVE);
-            command = "delete -n " + ASTERIX_INSTANCE_NAME;
-            cmdHandler.processCommand(command.split(" "));
-        }
-        instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(ASTERIX_INSTANCE_NAME);
-        assert (instance == null);
-    }
+		if (instance == null) {
+			return;
+		} else {
+			transformIntoRequiredState(State.INACTIVE);
+			command = "delete -n " + ASTERIX_INSTANCE_NAME;
+			cmdHandler.processCommand(command.split(" "));
+		}
+		instance = ServiceProvider.INSTANCE.getLookupService()
+				.getAsterixInstance(ASTERIX_INSTANCE_NAME);
+		assert (instance == null);
+	}
 
-    public static String getManagixHome() {
-        return managixHome;
-    }
+	public static String getManagixHome() {
+		return managixHome;
+	}
 
-    public static void installLibrary(String libraryName, String libraryDataverse, String libraryPath) throws Exception {
-        transformIntoRequiredState(State.INACTIVE);
-        String command = "install -n " + ASTERIX_INSTANCE_NAME + " -d " + libraryDataverse + " -l " + libraryName
-                + " -p " + libraryPath;
-        cmdHandler.processCommand(command.split(" "));
-    }
+	public static void installLibrary(String libraryName,
+			String libraryDataverse, String libraryPath) throws Exception {
+		transformIntoRequiredState(State.INACTIVE);
+		String command = "install -n " + ASTERIX_INSTANCE_NAME + " -d "
+				+ libraryDataverse + " -l " + libraryName + " -p "
+				+ libraryPath;
+		cmdHandler.processCommand(command.split(" "));
+	}
 
-    public static void uninstallLibrary(String dataverseName, String libraryName) throws Exception {
-        transformIntoRequiredState(State.INACTIVE);
-        String command = "uninstall -n " + ASTERIX_INSTANCE_NAME + " -d " + dataverseName + " -l " + "libraryName";
-        cmdHandler.processCommand(command.split(" "));
-    }
+	public static void uninstallLibrary(String dataverseName, String libraryName)
+			throws Exception {
+		transformIntoRequiredState(State.INACTIVE);
+		String command = "uninstall -n " + ASTERIX_INSTANCE_NAME + " -d "
+				+ dataverseName + " -l " + "libraryName";
+		cmdHandler.processCommand(command.split(" "));
+	}
 
-    public static void executeCommand(String command) throws Exception {
-        cmdHandler.processCommand(command.trim().split(" "));
-    }
+	public static void executeCommand(String command) throws Exception {
+		cmdHandler.processCommand(command.trim().split(" "));
+	}
 
 }
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
index c8f6cee..421eb42 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
@@ -25,12 +25,12 @@
 import org.junit.Test;
 import org.junit.runners.Parameterized.Parameters;
 
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
 import edu.uci.ics.asterix.installer.command.CommandHandler;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
 import edu.uci.ics.asterix.test.aql.TestsUtils;
 import edu.uci.ics.asterix.testframework.context.TestCaseContext;
 
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
new file mode 100644
index 0000000..2b3ce0d
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
@@ -0,0 +1,28 @@
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+create type TwitterUserType as closed {
+	screen-name: string,
+	lang: string,
+	friends_count: int32,
+	statuses_count: int32,
+	name: string,
+	followers_count: int32
+} 
+
+create type TweetMessageType as closed {
+	tweetid: string,
+        user: TwitterUserType,
+        sender-location: point,
+	send-time: datetime,
+        referred-topics: {{ string }},
+	message-text: string
+}
+
+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
new file mode 100644
index 0000000..64dbf25
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
@@ -0,0 +1,3 @@
+use dataverse feeds;
+
+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.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql
new file mode 100644
index 0000000..c5da56a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql
@@ -0,0 +1 @@
+40000
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
new file mode 100644
index 0000000..d03b9fe
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
@@ -0,0 +1,10 @@
+use dataverse feeds;
+
+let $numTuples:=count(for $x in dataset Tweets
+return $x)
+let $result:=if($numTuples > 225)
+then 
+ 1
+else
+ 0
+return $result
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm
@@ -0,0 +1 @@
+1
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
@@ -0,0 +1 @@
+1
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml b/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml
new file mode 100644
index 0000000..0d9ed23
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml
@@ -0,0 +1,10 @@
+<test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
+  <test-group name="fault-tolerance">
+    <test-case FilePath="feeds">
+      <compilation-unit name="IN1-cluster-restart">
+        <output-dir compare="Text">IN1-cluster-restart</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql
new file mode 100644
index 0000000..edd08ce
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  The feed simulator simulates feed from a file in the local fs.
+                  Associate with the feed an external user-defined function. The UDF 
+                  finds topics in each tweet. A topic is identified by a #. 
+                  Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date         : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+create type TweetType as closed {
+  id: string,
+  username : string,
+  location : string,
+  text : string,
+  timestamp : string,
+  topics : {{string}}?
+}
+
+create feed dataset TweetFeed(TweetType)
+using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
+(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://../asterix-app/data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
+apply function testlib#parseTweet@1
+primary key id;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql
new file mode 100644
index 0000000..b7e03ec
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  The feed simulator simulates feed from a file in the local fs.
+                  Associate with the feed an external user-defined function. The UDF 
+                  finds topics in each tweet. A topic is identified by a #. 
+                  Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date         : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+begin feed TweetFeed;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql
new file mode 100644
index 0000000..fd36fbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  The feed simulator simulates feed from a file in the local fs.
+                  Associate with the feed an external user-defined function. The UDF 
+                  finds topics in each tweet. A topic is identified by a #. 
+                  Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date         : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+for $x in dataset TweetFeed
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql
new file mode 100644
index 0000000..e140d9a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+create type CountryCapitalType if not exists as closed {
+country: string,
+capital: string
+};
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql
new file mode 100644
index 0000000..256a3cd
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql
@@ -0,0 +1,5 @@
+use dataverse externallibtest;
+
+let $input:=["England","Italy","China","United States","India","Jupiter"]
+for $country in $input
+return testlib#getCapital($country)
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql
new file mode 100644
index 0000000..11a5ddc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql
@@ -0,0 +1,9 @@
+use dataverse externallibtest;
+
+create type TextType if not exists as closed {
+id: int32,
+text: string
+};
+
+create dataset Check(TextType)
+primary key id;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql
new file mode 100644
index 0000000..8a14669
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+insert into dataset Check (
+{"id": 1, "text":"university of california, irvine"}
+);
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql
new file mode 100644
index 0000000..36f3133
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql
@@ -0,0 +1,7 @@
+use dataverse externallibtest;
+
+insert into dataset Check (
+  for $x in dataset Check
+  let $y:=testlib#toUpper($x)
+  return $y
+);
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql
new file mode 100644
index 0000000..997c333
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+for $x in  dataset Check 
+where $x.id < 0
+order by $x.id
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql
new file mode 100644
index 0000000..ce255b8
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql
@@ -0,0 +1,4 @@
+use dataverse externallibtest;
+
+let $x:=testlib#mysum(3,4)
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql
new file mode 100644
index 0000000..67635f5
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql
@@ -0,0 +1,7 @@
+use dataverse externallibtest;
+
+create type TextType if not exists as closed {
+id: int32,
+text: string
+};
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql
new file mode 100644
index 0000000..a742203
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql
@@ -0,0 +1,5 @@
+use dataverse externallibtest;
+
+let $input:={"id": 1, "text":"university of california, irvine"}
+let $x:=testlib#toUpper($input)
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql
new file mode 100644
index 0000000..40316d8
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Dataverse
+order by $x.DataverseName
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql
new file mode 100644
index 0000000..fc47972
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Function
+order by $x.Name
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql
new file mode 100644
index 0000000..36a8a52
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Library
+order by $x.Name
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm
new file mode 100644
index 0000000..16e9591
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm
@@ -0,0 +1,6 @@
+{ "country": "England", "capital": "London" }
+{ "country": "Italy", "capital": "Rome" }
+{ "country": "China", "capital": "Beijing" }
+{ "country": "United States", "capital": "Washington D.C." }
+{ "country": "India", "capital": "New Delhi" }
+{ "country": "Jupiter", "capital": "NOT_FOUND" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm
new file mode 100644
index 0000000..a839cbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm
@@ -0,0 +1 @@
+{ "id": -1, "text": "UNIVERSITY OF CALIFORNIA, IRVINE" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm
new file mode 100644
index 0000000..7f8f011
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm
@@ -0,0 +1 @@
+7
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm
new file mode 100644
index 0000000..a839cbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm
@@ -0,0 +1 @@
+{ "id": -1, "text": "UNIVERSITY OF CALIFORNIA, IRVINE" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm
new file mode 100644
index 0000000..330c347
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "Metadata", "DataFormat": "edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 25 11:17:56 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "externallibtest", "DataFormat": "edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 25 11:18:12 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
new file mode 100644
index 0000000..bb42e62
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
@@ -0,0 +1,12 @@
+{ "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "edu.uci.ics.asterix.external.library.AllTypesFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "edu.uci.ics.asterix.external.library.EchoDelayFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "edu.uci.ics.asterix.external.library.CapitalFinderFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "edu.uci.ics.asterix.external.library.SumFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetType" ], "ReturnType": "TweetType", "Definition": "edu.uci.ics.asterix.external.library.ParseTweetFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "edu.uci.ics.asterix.external.library.UpperCaseFactory
+			", "Language": "JAVA", "Kind": "SCALAR" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm
new file mode 100644
index 0000000..573db0c
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "externallibtest", "Name": "testlib", "Timestamp": "Mon Apr 22 23:36:55 PDT 2013" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/testsuite.xml b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
new file mode 100644
index 0000000..f97c050
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
@@ -0,0 +1,42 @@
+<test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
+  <test-group name="library-functions">
+    <test-case FilePath="library-functions">
+      <compilation-unit name="mysum">
+        <output-dir compare="Text">mysum</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="library-functions">
+      <compilation-unit name="toUpper">
+        <output-dir compare="Text">toUpper</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="library-functions">
+      <compilation-unit name="insert-from-select">
+        <output-dir compare="Text">insert-from-select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="library-functions">
+      <compilation-unit name="getCapital">
+        <output-dir compare="Text">getCapital</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="library-metadata">
+    <test-case FilePath="library-metadata">
+      <compilation-unit name="functionDataset">
+        <output-dir compare="Text">functionDataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="library-metadata">
+      <compilation-unit name="libraryDataset">
+        <output-dir compare="Text">libraryDataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="library-metadata">
+      <compilation-unit name="dataverseDataset">
+        <output-dir compare="Text">dataverseDataset</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
+
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql
new file mode 100644
index 0000000..97ad91e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql
@@ -0,0 +1 @@
+install -n asterix -d externallibtest -l testlib -p ../asterix-external-data/target/testlib-zip-binary-assembly.zip
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql
new file mode 100644
index 0000000..5a46092
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql
@@ -0,0 +1,2 @@
+for $x in dataset Metadata.Library
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql
new file mode 100644
index 0000000..1b5ae40
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql
@@ -0,0 +1 @@
+uninstall -n asterix -d externallibtest -l testlib 
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql
new file mode 100644
index 0000000..5a46092
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql
@@ -0,0 +1,2 @@
+for $x in dataset Metadata.Library
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm
new file mode 100644
index 0000000..a5d5c9b
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "externallibtest", "Name": "testlib", "Timestamp": "Wed Apr 24 17:25:25 PDT 2013" }
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.adm
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
copy to asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.adm
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml b/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
index f1949ca..969da43 100644
--- a/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
@@ -19,6 +19,16 @@
         <output-dir compare="Text">backupRestore</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="asterix-lifecycle">
+      <compilation-unit name="installLibrary">
+        <output-dir compare="Text">installLibrary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="asterix-lifecycle">
+      <compilation-unit name="uninstallLibrary">
+        <output-dir compare="Text">uninstallLibrary</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
 
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 6b1e8cf..febe440 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -51,13 +47,13 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-external-data</artifactId>
+			<artifactId>asterix-runtime</artifactId>
 			<version>0.8.1-SNAPSHOT</version>
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-runtime</artifactId>
+			<artifactId>asterix-events</artifactId>
 			<version>0.8.1-SNAPSHOT</version>
 			<scope>compile</scope>
 		</dependency>
@@ -80,6 +76,12 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
 		</dependency>
+                <dependency>
+                        <groupId>edu.uci.ics.hyracks</groupId>
+                        <artifactId>hyracks-hdfs-core</artifactId>
+                        <version>${hyracks.version}</version>
+                </dependency>
+
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
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 91be90b..f78eb31 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
@@ -27,9 +27,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.FeedConnectionId;
 
 /**
  * Caches metadata entities such that the MetadataManager does not have to
@@ -38,6 +43,7 @@
  * simply ignored, i.e., updates are not not applied to the cache.
  */
 public class MetadataCache {
+
     // Key is dataverse name.
     protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
     // Key is dataverse name. Key of value map is dataset name.
@@ -52,6 +58,15 @@
     protected final Map<FunctionSignature, Function> functions = new HashMap<FunctionSignature, Function>();
     // 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<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 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>>();
     // Key is DataverseName, Key of the value map is the Policy name   
     protected final Map<String, Map<String, CompactionPolicy>> compactionPolicies = new HashMap<String, Map<String, CompactionPolicy>>();
 
@@ -89,15 +104,21 @@
                         synchronized (datatypes) {
                             synchronized (functions) {
                                 synchronized (adapters) {
-                                    synchronized (compactionPolicies) {
-                                        dataverses.clear();
-                                        nodeGroups.clear();
-                                        datasets.clear();
-                                        indexes.clear();
-                                        datatypes.clear();
-                                        functions.clear();
-                                        adapters.clear();
-                                        compactionPolicies.clear();
+                                    synchronized (feedActivity) {
+                                        synchronized (libraries) {
+                                            synchronized (compactionPolicies) {
+                                                dataverses.clear();
+                                                nodeGroups.clear();
+                                                datasets.clear();
+                                                indexes.clear();
+                                                datatypes.clear();
+                                                functions.clear();
+                                                adapters.clear();
+                                                feedActivity.clear();
+                                                libraries.clear();
+                                                compactionPolicies.clear();
+                                            }
+                                        }
                                     }
                                 }
                             }
@@ -212,22 +233,43 @@
                 synchronized (indexes) {
                     synchronized (datatypes) {
                         synchronized (functions) {
-                            synchronized (compactionPolicies) {
-                                datasets.remove(dataverse.getDataverseName());
-                                indexes.remove(dataverse.getDataverseName());
-                                datatypes.remove(dataverse.getDataverseName());
-                                adapters.remove(dataverse.getDataverseName());
-                                compactionPolicies.remove(dataverse.getDataverseName());
-                                List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
-                                for (FunctionSignature signature : functions.keySet()) {
-                                    if (signature.getNamespace().equals(dataverse.getDataverseName())) {
-                                        markedFunctionsForRemoval.add(signature);
+                            synchronized (adapters) {
+                                synchronized (libraries) {
+                                    synchronized (feedActivity) {
+                                        synchronized (feeds) {
+                                            synchronized (compactionPolicies) {
+                                                datasets.remove(dataverse.getDataverseName());
+                                                indexes.remove(dataverse.getDataverseName());
+                                                datatypes.remove(dataverse.getDataverseName());
+                                                adapters.remove(dataverse.getDataverseName());
+                                                compactionPolicies.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);
+                                                }
+
+                                                libraries.remove(dataverse.getDataverseName());
+                                                feeds.remove(dataverse.getDataverseName());
+
+                                                return dataverses.remove(dataverse.getDataverseName());
+                                            }
+                                        }
                                     }
                                 }
-                                for (FunctionSignature signature : markedFunctionsForRemoval) {
-                                    functions.remove(signature);
-                                }
-                                return dataverses.remove(dataverse.getDataverseName());
                             }
                         }
                     }
@@ -409,6 +451,32 @@
         }
     }
 
+    public Object addFeedPolicyIfNotExists(FeedPolicy feedPolicy) {
+        synchronized (feedPolicy) {
+            Map<String, FeedPolicy> p = feedPolicies.get(feedPolicy.getDataverseName());
+            if (p == null) {
+                p = new HashMap<String, FeedPolicy>();
+                p.put(feedPolicy.getPolicyName(), feedPolicy);
+                feedPolicies.put(feedPolicy.getDataverseName(), p);
+            } else {
+                if (p.get(feedPolicy.getPolicyName()) == null) {
+                    p.put(feedPolicy.getPolicyName(), feedPolicy);
+                }
+            }
+            return null;
+        }
+    }
+
+    public Object dropFeedPolicy(FeedPolicy feedPolicy) {
+        synchronized (feedPolicies) {
+            Map<String, FeedPolicy> p = feedPolicies.get(feedPolicy.getDataverseName());
+            if (p != null && p.get(feedPolicy.getPolicyName()) != null) {
+                return p.remove(feedPolicy).getPolicyName();
+            }
+            return null;
+        }
+    }
+
     public Object addAdapterIfNotExists(DatasourceAdapter adapter) {
         synchronized (adapters) {
             Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
@@ -435,4 +503,61 @@
             return null;
         }
     }
+
+    public Object addFeedActivityIfNotExists(FeedActivity fa) {
+        synchronized (feedActivity) {
+            FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
+            if (!feedActivity.containsKey(fid)) {
+                feedActivity.put(fid, fa);
+            }
+        }
+        return null;
+    }
+
+    public Object dropFeedActivity(FeedActivity fa) {
+        synchronized (feedActivity) {
+            FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
+            return feedActivity.remove(fid);
+        }
+    }
+
+    public Object addLibraryIfNotExists(Library library) {
+        synchronized (libraries) {
+            Map<String, Library> libsInDataverse = libraries.get(library.getDataverseName());
+            boolean needToAddd = (libsInDataverse == null || libsInDataverse.get(library.getName()) != null);
+            if (needToAddd) {
+                if (libsInDataverse == null) {
+                    libsInDataverse = new HashMap<String, Library>();
+                    libraries.put(library.getDataverseName(), libsInDataverse);
+                }
+                return libsInDataverse.put(library.getDataverseName(), library);
+            }
+            return null;
+        }
+    }
+
+    public Object dropLibrary(Library library) {
+        synchronized (libraries) {
+            Map<String, Library> librariesInDataverse = libraries.get(library.getDataverseName());
+            if (librariesInDataverse != null) {
+                return librariesInDataverse.remove(library.getName());
+            }
+            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/MetadataException.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
index b606839..e58466d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
@@ -17,6 +17,7 @@
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 
+
 public class MetadataException extends AsterixException {
     private static final long serialVersionUID = 1L;
 
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 46159ba..590890a 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
@@ -32,11 +32,18 @@
 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;
 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.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+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;
 
 /**
  * Provides access to Asterix metadata via remote methods to the metadata node.
@@ -71,6 +78,7 @@
  * with transaction ids of regular jobs or other metadata transactions.
  */
 public class MetadataManager implements IMetadataManager {
+
     // Set in init().
     public static MetadataManager INSTANCE;
     private final MetadataCache cache = new MetadataCache();
@@ -78,6 +86,7 @@
     private IMetadataNode metadataNode;
     private final ReadWriteLock metadataLatch;
     private final AsterixMetadataProperties metadataProperties;
+    private IHyracksClientConnection hcc;
 
     public MetadataManager(IAsterixStateProxy proxy, AsterixMetadataProperties metadataProperties) {
         if (proxy == null) {
@@ -96,10 +105,14 @@
             if (metadataNode != null) {
                 return;
             }
-            metadataNode = proxy.getMetadataNode();
-            if (metadataNode == null) {
-                throw new Error("Failed to get the MetadataNode.\n" + "The MetadataNode was configured to run on NC: "
-                        + metadataProperties.getMetadataNodeName());
+            while (metadataNode == null) {
+                metadataNode = proxy.getMetadataNode();
+                try {
+                    Thread.sleep(500);
+                } catch (InterruptedException ie) {
+                    throw new RemoteException("Interrupted while waiting for obtaining handle to Metadata node " + "("
+                            + metadataProperties.getMetadataNodeName() + ")");
+                }
             }
         }
     }
@@ -555,6 +568,16 @@
     }
 
     @Override
+    public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicy feedPolicy) throws MetadataException {
+        try {
+            metadataNode.addFeedPolicy(mdTxnCtx.getJobId(), feedPolicy);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        mdTxnCtx.addFeedPolicy(feedPolicy);
+    }
+
+    @Override
     public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
         try {
             metadataNode.initializeDatasetIdFactory(ctx.getJobId());
@@ -610,6 +633,77 @@
     }
 
     @Override
+    public void registerFeedActivity(MetadataTransactionContext ctx, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException {
+        try {
+            metadataNode.registerFeedActivity(ctx.getJobId(), feedId, feedActivity);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
+            FeedActivityType... feedActivityTypes) throws MetadataException {
+
+        FeedActivity feedActivity = null;
+        try {
+            feedActivity = metadataNode.getRecentFeedActivity(ctx.getJobId(), feedId, feedActivityTypes);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return feedActivity;
+    }
+
+    public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+            throws MetadataException {
+        try {
+            metadataNode.dropLibrary(ctx.getJobId(), dataverseName, libraryName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.dropLibrary(dataverseName, libraryName);
+    }
+
+    @Override
+    public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
+            throws MetadataException {
+        List<Library> dataverseLibaries = null;
+        try {
+            // Assuming that the transaction can read its own writes on the
+            // metadata node.
+            dataverseLibaries = metadataNode.getDataverseLibraries(ctx.getJobId(), dataverseName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        // Don't update the cache to avoid checking against the transaction's
+        // uncommitted functions.
+        return dataverseLibaries;
+    }
+
+    @Override
+    public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException {
+        try {
+            metadataNode.addLibrary(ctx.getJobId(), library);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.addLibrary(library);
+    }
+
+    @Override
+    public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+            throws MetadataException, RemoteException {
+        Library library = null;
+        try {
+            library = metadataNode.getLibrary(ctx.getJobId(), dataverseName, libraryName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return library;
+    }
+
+    @Override
     public void acquireWriteLatch() {
         metadataLatch.writeLock().lock();
     }
@@ -628,4 +722,72 @@
     public void releaseReadLatch() {
         metadataLatch.readLock().unlock();
     }
+
+    @Override
+    public FeedPolicy getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+            throws MetadataException {
+
+        FeedPolicy FeedPolicy = null;
+        try {
+            FeedPolicy = metadataNode.getFeedPolicy(ctx.getJobId(), dataverse, policyName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return FeedPolicy;
+    }
+
+    @Override
+    public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx, String dataverse, String dataset)
+            throws MetadataException {
+        List<FeedActivity> feedActivities = null;
+        try {
+            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);
+        }
+        return feedActivities;
+    }
+
 }
\ No newline at end of file
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 a3f736e..3c7e4a3 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
@@ -17,7 +17,12 @@
 
 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;
@@ -42,9 +47,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.FeedActivity.FeedActivityType;
+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.InternalDatasetDetails;
+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.entitytupletranslators.CompactionPolicyTupleTranslator;
@@ -52,10 +62,16 @@
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
 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.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;
@@ -166,7 +182,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(),
@@ -305,13 +321,15 @@
     @Override
     public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
         try {
-            List<Dataset> dataverseDatasets;
 
+            List<Dataset> dataverseDatasets;
+            Dataset ds;
             dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
             if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
                 // Drop all datasets in this dataverse.
                 for (int i = 0; i < dataverseDatasets.size(); i++) {
-                    dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+                    ds = dataverseDatasets.get(i);
+                    dropDataset(jobId, dataverseName, ds.getDatasetName());
                 }
             }
             List<Datatype> dataverseDatatypes;
@@ -345,6 +363,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
@@ -388,7 +417,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
@@ -416,7 +445,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) {
@@ -424,6 +453,7 @@
                     }
                 }
             }
+
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -644,6 +674,34 @@
         }
     }
 
+    @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 {
+            ITupleReference searchKey = createTuple(dataverseName);
+            LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
+            IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
+            List<Library> results = new ArrayList<Library>();
+            searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+            return results;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
     private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
@@ -778,6 +836,7 @@
             }
             return results.get(0);
         } catch (Exception e) {
+            e.printStackTrace();
             throw new MetadataException(e);
         }
     }
@@ -839,9 +898,9 @@
                     + functionSignature.getArity());
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+            ITupleReference functionTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
                     searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -1177,7 +1236,321 @@
     }
 
     @Override
+    public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException {
+        try {
+            // Insert into the 'Library' dataset.
+            LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(true);
+            ITupleReference libraryTuple = tupleReaderWriter.getTupleFromMetadataEntity(library);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
+
+        } catch (TreeIndexException e) {
+            throw new MetadataException("A library with this name " + library.getDataverseName()
+                    + " already exists in dataverse '" + library.getDataverseName() + "'.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+            RemoteException {
+        Library library;
+        try {
+            library = getLibrary(jobId, dataverseName, libraryName);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+        if (library == null) {
+            throw new MetadataException("Cannot drop library '" + library + "' because it doesn't exist.");
+        }
+        try {
+            // Delete entry from the 'Library' dataset.
+            ITupleReference searchKey = createTuple(dataverseName, libraryName);
+            // Searches the index for the tuple to be deleted. Acquires an S
+            // lock on the 'Adapter' dataset.
+            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
+
+            // TODO: Change this to be a BTree specific exception, e.g.,
+            // BTreeKeyDoesNotExistException.
+        } catch (TreeIndexException e) {
+            throw new MetadataException("Cannot drop library '" + libraryName, e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+            RemoteException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, libraryName);
+            LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
+            List<Library> results = new ArrayList<Library>();
+            IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+            if (results.isEmpty()) {
+                return null;
+            }
+            return results.get(0);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
     public int getMostRecentDatasetId() throws MetadataException, RemoteException {
         return DatasetIdFactory.getMostRecentDatasetId();
     }
+
+    @Override
+    public void registerFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException, RemoteException {
+        try {
+            if (!FeedActivityIdFactory.isInitialized()) {
+                initializeFeedActivityIdFactory(jobId);
+            }
+            feedActivity.setActivityId(FeedActivityIdFactory.generateFeedActivityId());
+            FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+            ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(feedActivity);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, tuple);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivityType... activityType)
+            throws MetadataException, RemoteException {
+        try {
+            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>(
+                    tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+            if (!results.isEmpty()) {
+                Collections.sort(results);
+                if (activityType == null) {
+                    return results.get(0);
+                } else {
+                    for (FeedActivity result : results) {
+                        for (FeedActivityType ft : activityType) {
+                            if (result.getActivityType().equals(ft)) {
+                                return result;
+                            }
+                        }
+                    }
+                }
+            }
+            return null;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public void initializeFeedActivityIdFactory(JobId jobId) throws MetadataException, RemoteException {
+        try {
+            ITupleReference searchKey = createTuple();
+            FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+            List<FeedActivity> results = new ArrayList<FeedActivity>();
+            IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+                    tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+            int maxActivityId = 0;
+            for (FeedActivity fa : results) {
+                if (maxActivityId < fa.getActivityId()) {
+                    maxActivityId = fa.getActivityId();
+                }
+            }
+            FeedActivityIdFactory.initialize(maxActivityId);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    public void addFeedPolicy(JobId jobId, FeedPolicy feedPolicy) throws MetadataException, RemoteException {
+        try {
+            // Insert into the 'FeedPolicy' dataset.
+            FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(true);
+            ITupleReference feedPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedPolicy);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
+
+        } catch (TreeIndexException e) {
+            throw new MetadataException("A feed policy with this name " + feedPolicy.getPolicyName()
+                    + " already exists in dataverse '" + feedPolicy.getPolicyName() + "'.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    public FeedPolicy getFeedPolicy(JobId jobId, String dataverse, String policyName) throws MetadataException,
+            RemoteException {
+
+        try {
+            ITupleReference searchKey = createTuple(dataverse, policyName);
+            FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(false);
+            List<FeedPolicy> results = new ArrayList<FeedPolicy>();
+            IValueExtractor<FeedPolicy> valueExtractor = new MetadataEntityValueExtractor<FeedPolicy>(tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
+            if (!results.isEmpty()) {
+                return results.get(0);
+            }
+            return null;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public List<FeedActivity> getActiveFeeds(JobId jobId, String dataverse, String dataset) throws MetadataException,
+            RemoteException {
+        List<FeedActivity> activeFeeds = new ArrayList<FeedActivity>();
+        Map<FeedConnectionId, FeedActivity> aFeeds = new HashMap<FeedConnectionId, FeedActivity>();
+        boolean invalidArgs = (dataverse == null && dataset != null);
+        if (invalidArgs) {
+            throw new MetadataException("Invalid arguments " + dataverse + " " + dataset);
+        }
+        try {
+            ITupleReference searchKey = createTuple();
+            FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+            List<FeedActivity> results = new ArrayList<FeedActivity>();
+            IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+                    tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+            Collections.sort(results); // recent activity first
+            FeedConnectionId fid = null;
+            Set<FeedConnectionId> terminatedFeeds = new HashSet<FeedConnectionId>();
+            for (FeedActivity fa : results) {
+                if (dataverse != null) {
+                    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_BEGIN:
+                        if (!terminatedFeeds.contains(fid)) {
+                            if (aFeeds.get(fid) == null || fa.getActivityId() > aFeeds.get(fid).getActivityId()) {
+                                aFeeds.put(fid, fa);
+                            }
+                        }
+                        break;
+                    case FEED_END:
+                        terminatedFeeds.add(fid);
+                        break;
+                    default: //ignore    
+                }
+            }
+            for (FeedActivity f : aFeeds.values()) {
+                System.out.println("ACTIVE FEEDS " + f.getFeedName());
+                activeFeeds.add(f);
+            }
+            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 (TreeIndexException 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:
+                            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 3705fa1..1c990af 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,19 +16,23 @@
 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;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
 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.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;
+import edu.uci.ics.asterix.metadata.entities.Library;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
@@ -130,6 +134,11 @@
         logAndApply(new MetadataLogicalOperation(dataverse, false));
     }
 
+    public void addLibrary(Library library) {
+        droppedCache.dropLibrary(library);
+        logAndApply(new MetadataLogicalOperation(library, true));
+    }
+
     public void dropDataDatatype(String dataverseName, String datatypeName) {
         Datatype datatype = new Datatype(dataverseName, datatypeName, null, false);
         droppedCache.addDatatypeIfNotExists(datatype);
@@ -156,6 +165,12 @@
         logAndApply(new MetadataLogicalOperation(adapter, false));
     }
 
+    public void dropLibrary(String dataverseName, String libraryName) {
+        Library library = new Library(dataverseName, libraryName);
+        droppedCache.addLibraryIfNotExists(library);
+        logAndApply(new MetadataLogicalOperation(library, false));
+    }
+
     public void logAndApply(MetadataLogicalOperation op) {
         opLog.add(op);
         doOperation(op);
@@ -201,10 +216,29 @@
         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();
         droppedCache.clear();
         opLog.clear();
     }
+
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
new file mode 100644
index 0000000..049a45c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.asterix.metadata.api;
+
+/*
+ * 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.
+ */
+import java.util.Set;
+
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+
+public interface IClusterEventsSubscriber {
+
+    /**
+     * @param deadNodeIds
+     * @return
+     */
+    public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds);
+
+    /**
+     * @param joinedNodeId
+     * @return
+     */
+    public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId);
+
+    /**
+     * @param response
+     */
+    public void notifyRequestCompletion(IClusterManagementWorkResponse response);
+
+    /**
+     * @param previousState
+     * @param newState
+     */
+    public void notifyStateChange(State previousState, State newState);
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
similarity index 68%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
copy to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
index dcef2c8..65ac354 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
@@ -12,17 +12,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.api;
 
-import java.io.Serializable;
+public interface IClusterManagementWork {
 
-public interface IFeedMessage extends Serializable {
-
-    public enum MessageType {
-        STOP,
-        ALTER,
+    public enum WorkType {
+        ADD_NODE,
+        REMOVE_NODE
     }
 
-    public MessageType getMessageType();
+    public WorkType getClusterManagementWorkType();
 
+    public int getWorkId();
+
+    public IClusterEventsSubscriber getSourceSubscriber();
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java
new file mode 100644
index 0000000..ea07a62
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.asterix.metadata.api;
+
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+
+public interface IClusterManager {
+
+    /**
+     * @param node
+     * @throws AsterixException
+     */
+    public void addNode(Node node) throws AsterixException;
+
+    /**
+     * @param node
+     * @throws AsterixException
+     */
+    public void removeNode(Node node) throws AsterixException;
+
+    /**
+     * @param subscriber
+     */
+    public void registerSubscriber(IClusterEventsSubscriber subscriber);
+
+    /**
+     * @param sunscriber
+     * @return
+     */
+    public boolean deregisterSubscriber(IClusterEventsSubscriber sunscriber);
+
+    /**
+     * @return
+     */
+    public Set<IClusterEventsSubscriber> getRegisteredClusterEventSubscribers();
+
+}
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 392c8a1..f23275c 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
@@ -8,7 +8,7 @@
  * 
  * 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.
+ * 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.
  */
@@ -27,10 +27,16 @@
 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;
 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.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 
 /**
  * A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -458,6 +464,79 @@
     public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws MetadataException;
 
+    /**
+     * @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, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param dataverseName
+     * @param datasetName
+     * @return
+     * @throws MetadataException
+     */
+    public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
+            FeedActivityType... activityTypeFilter) throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param policy
+     * @throws MetadataException
+     */
+    public void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicy policy) throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param dataverse
+     * @param policyName
+     * @return
+     * @throws MetadataException
+     */
+    public FeedPolicy getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+            throws MetadataException;
+
+    /**
+     * @param ctx
+     * @param dataverse
+     * @param dataset
+     * @return
+     * @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;
 
     public int getMostRecentDatasetId() throws MetadataException;
@@ -470,4 +549,62 @@
 
     public void releaseReadLatch();
 
+    /**
+     * Removes a library , acquiring local locks on behalf of the given
+     * transaction id.
+     * 
+     * @param ctx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the adapter that is to be deleted.
+     * @param libraryName
+     *            Name of library to be deleted. MetadataException for example,
+     *            if the library does not exists.
+     * @throws RemoteException
+     */
+    public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+            throws MetadataException;
+
+    /**
+     * Adds a library, acquiring local locks on behalf of the given
+     * transaction id.
+     * 
+     * @param ctx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param library
+     *            Library to be added
+     * @throws MetadataException
+     *             for example, if the library is already added.
+     * @throws RemoteException
+     */
+    public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException;
+
+    /**
+     * @param ctx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the library that is to be retrieved.
+     * @param libraryName
+     *            name of the library that is to be retrieved
+     * @return Library
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+            throws MetadataException, RemoteException;
+
+    /**
+     * Retireve libraries installed in a given dataverse.
+     * 
+     * @param ctx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the library that is to be retrieved.
+     * @return Library
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
+            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 f27268f..8434847 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
@@ -29,10 +29,16 @@
 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;
 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.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
 
 /**
  * A metadata node stores metadata in its local storage structures (currently
@@ -482,8 +488,19 @@
     public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException,
             RemoteException;
 
+    public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId,
+            FeedActivityType... feedActivityFilter) throws MetadataException, RemoteException;
+
     /**
      * @param jobId
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public void initializeFeedActivityIdFactory(JobId jobId) throws MetadataException, RemoteException;
+
+    /**
+     * 
+     * @param jobId
      * @param dataverse
      * @param policy
      * @return
@@ -493,8 +510,165 @@
     public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
             RemoteException;
 
+    /**
+     * 
+     * @param jobId
+     * @throws MetadataException
+     * @throws RemoteException
+     */
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
 
+    /**
+     * 
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
     public int getMostRecentDatasetId() throws MetadataException, RemoteException;
 
+    /**
+     * @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, FeedConnectionId feedId, FeedActivity feedActivity)
+            throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
+     * @param feedPolicy
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public void addFeedPolicy(JobId jobId, FeedPolicy feedPolicy) throws MetadataException, RemoteException;
+
+    /**
+     * @param jobId
+     * @param dataverse
+     * @param policy
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public FeedPolicy getFeedPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
+            RemoteException;
+
+    /**
+     * @param jobId
+     * @param dataverse
+     * @param dataset
+     * @return
+     * @throws MetadataException
+     * @throws 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
+     * transaction id.
+     * 
+     * @param txnId
+     *            A globally unique id for an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the adapter that is to be deleted.
+     * @param libraryName
+     *            Name of library to be deleted. MetadataException for example,
+     *            if the library does not exists.
+     * @throws RemoteException
+     */
+    public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+            RemoteException;
+
+    /**
+     * Adds a library, acquiring local locks on behalf of the given
+     * transaction id.
+     * 
+     * @param txnId
+     *            A globally unique id for an active metadata transaction.
+     * @param library
+     *            Library to be added
+     * @throws MetadataException
+     *             for example, if the library is already added.
+     * @throws RemoteException
+     */
+    public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException;
+
+    /**
+     * @param txnId
+     *            A globally unique id for an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the library that is to be retrieved.
+     * @param libraryName
+     *            name of the library that is to be retrieved
+     * @return Library
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+            RemoteException;
+
+    /**
+     * Retireve libraries installed in a given dataverse.
+     * 
+     * @param txnId
+     *            A globally unique id for an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the library that is to be retrieved.
+     * @return Library
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    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 455715d..0b2a6c7 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
@@ -37,8 +37,6 @@
 import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.IDatasetDetails;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -51,12 +49,16 @@
 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.FeedPolicy;
 import edu.uci.ics.asterix.metadata.entities.Index;
 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.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
@@ -119,7 +121,10 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET };
+                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
+                MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
+                MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET };
+
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -171,6 +176,7 @@
                 insertNodes(mdTxnCtx);
                 insertInitialGroups(mdTxnCtx);
                 insertInitialAdapters(mdTxnCtx);
+                insertInitialFeedPolicies(mdTxnCtx);
                 insertInitialCompactionPolicies(mdTxnCtx);
 
                 if (LOGGER.isLoggable(Level.INFO)) {
@@ -310,7 +316,11 @@
                 "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory",
                 "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.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" };
         DatasourceAdapter adapter;
         for (String adapterClassName : builtInAdapterClassNames) {
             adapter = getAdapter(adapterClassName);
@@ -318,6 +328,12 @@
         }
     }
 
+    private static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
+        for (FeedPolicy feedPolicy : BuiltinFeedPolicies.policies) {
+            MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, feedPolicy);
+        }
+    }
+  
     private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
         String[] builtInCompactionPolicyClassNames = new String[] {
                 "edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory",
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
index 4c9b7d2..1957666 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
@@ -19,10 +19,10 @@
 
     // Name of the dataverse the metadata lives in.
     public final static String METADATA_DATAVERSE_NAME = "Metadata";
-    
+
     // Name of the node group where metadata is stored on.
     public final static String METADATA_NODEGROUP_NAME = "MetadataGroup";
-    
+
     // Name of the default nodegroup where internal/feed datasets will be partitioned
     // if an explicit nodegroup is not specified at the time of creation of a dataset
     public static final String METADATA_DEFAULT_NODEGROUP_NAME = "DEFAULT_NG_ALL_NODES";
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 f6bc9e6..68f9a74 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
@@ -32,6 +32,13 @@
     public static IMetadataIndex NODE_DATASET;
     public static IMetadataIndex NODEGROUP_DATASET;
     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;
+    public static IMetadataIndex COMPACTION_POLICY_DATASET;
+
     public static final int METADATA_DATASET_ID = 0;
     public static final int DATAVERSE_DATASET_ID = 1;
     public static final int DATASET_DATASET_ID = 2;
@@ -41,11 +48,15 @@
     public static final int NODEGROUP_DATASET_ID = 6;
     public static final int FUNCTION_DATASET_ID = 7;
     public static final int DATASOURCE_ADAPTER_DATASET_ID = 8;
-    public static final int COMPACTION_POLICY_DATASET_ID = 9;
+
+    public static final int LIBRARY_DATASET_ID = 9;
+    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 COMPACTION_POLICY_DATASET_ID = 13;
+
     public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
 
-    public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
-    public static IMetadataIndex COMPACTION_POLICY_DATASET;
 
     /**
      * Create all metadata primary index descriptors. MetadataRecordTypes must
@@ -95,6 +106,23 @@
                 MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_DATASET_ID, true, new int[] { 0,
                         1 });
 
+        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,
+                MetadataRecordTypes.LIBRARY_RECORDTYPE, LIBRARY_DATASET_ID, true, new int[] { 0, 1 });
+
+        FEED_POLICY_DATASET = new MetadataIndex("FeedPolicy", null, 3, new IAType[] { BuiltinType.ASTRING,
+                BuiltinType.ASTRING }, new String[] { "DataverseName", "PolicyName" }, 0,
+                MetadataRecordTypes.FEED_POLICY_RECORDTYPE, FEED_POLICY_DATASET_ID, true, new int[] { 0, 1 });
+
         COMPACTION_POLICY_DATASET = new MetadataIndex("CompactionPolicy", null, 3, new IAType[] { BuiltinType.ASTRING,
                 BuiltinType.ASTRING }, new String[] { "DataverseName", "CompactionPolicy" }, 0,
                 MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE, COMPACTION_POLICY_DATASET_ID, true,
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 8452340..4af3640 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
@@ -48,6 +48,13 @@
     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;
+    public static ARecordType FEED_ACTIVITY_DETAILS_RECORDTYPE;
+    public static ARecordType LIBRARY_RECORDTYPE;
     public static ARecordType COMPACTION_POLICY_RECORDTYPE;
 
     /**
@@ -58,6 +65,7 @@
         // depend on other types being created first.
         // These calls are one "dependency chain".
         try {
+            POLICY_PARAMS_RECORDTYPE = createPropertiesRecordType();
             DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE = createPropertiesRecordType();
             COMPACTION_POLICY_PROPERTIES_RECORDTYPE = createPropertiesRecordType();
             INTERNAL_DETAILS_RECORDTYPE = createInternalDetailsRecordType();
@@ -79,12 +87,33 @@
             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();
+            LIBRARY_RECORDTYPE = createLibraryRecordType();
+
             COMPACTION_POLICY_RECORDTYPE = createCompactionPolicyRecordType();
+
         } catch (AsterixException e) {
             throw new MetadataException(e);
         }
     }
 
+    public static final int FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+    public static final int FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX = 1;
+    public static final int FEED_POLICY_ARECORD_DESCRIPTION_FIELD_INDEX = 2;
+    public static final int FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX = 3;
+
+    private static ARecordType createFeedPolicyRecordType() throws AsterixException {
+        AUnorderedListType listPropertiesType = new AUnorderedListType(POLICY_PARAMS_RECORDTYPE, null);
+        String[] fieldNames = { "DataverseName", "PolicyName", "Description", "Properties" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, listPropertiesType };
+        return new ARecordType("FeedPolicyRecordType", fieldNames, fieldTypes, true);
+    }
+
     // Helper constants for accessing fields in an ARecord of type
     // DataverseRecordType.
     public static final int DATAVERSE_ARECORD_NAME_FIELD_INDEX = 0;
@@ -194,15 +223,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);
@@ -214,16 +242,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);
     }
 
@@ -382,4 +405,58 @@
         return new ARecordType("DatasourceAdapterRecordType", fieldNames, fieldTypes, true);
     }
 
+    // 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_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", "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;
+
+    private static ARecordType createLibraryRecordType() throws AsterixException {
+        String[] fieldNames = { "DataverseName", "Name", "Timestamp" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
+        return new ARecordType("LibraryRecordType", fieldNames, fieldTypes, true);
+    }
 }
\ No newline at end of file
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/cluster/AbstractClusterManagementWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AbstractClusterManagementWork.java
new file mode 100644
index 0000000..6948dbc
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AbstractClusterManagementWork.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cluster;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public abstract class AbstractClusterManagementWork implements IClusterManagementWork {
+
+    protected final IClusterEventsSubscriber subscriber;
+
+    protected final int workId;
+
+    @Override
+    public int getWorkId() {
+        return workId;
+    }
+
+    public AbstractClusterManagementWork(IClusterEventsSubscriber subscriber) {
+        this.subscriber = subscriber;
+        this.workId = WorkIdGenerator.getNextWorkId();
+    }
+
+    private static class WorkIdGenerator {
+        private static AtomicInteger workId = new AtomicInteger(0);
+
+        public static int getNextWorkId() {
+            return workId.incrementAndGet();
+        }
+
+    }
+
+    @Override
+    public IClusterEventsSubscriber getSourceSubscriber() {
+        return subscriber;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java
new file mode 100644
index 0000000..68dcc4c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+
+public class AddNodeWork extends AbstractClusterManagementWork {
+
+    private final int numberOfNodes;
+
+    @Override
+    public WorkType getClusterManagementWorkType() {
+        return WorkType.ADD_NODE;
+    }
+
+    public AddNodeWork(int numberOfNodes, IClusterEventsSubscriber subscriber) {
+        super(subscriber);
+        this.numberOfNodes = numberOfNodes;
+    }
+
+    public int getNumberOfNodes() {
+        return numberOfNodes;
+    }
+
+    @Override
+    public String toString() {
+        return WorkType.ADD_NODE + " " + numberOfNodes + " requested by " + subscriber;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java
new file mode 100644
index 0000000..40999f0
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cluster;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AddNodeWorkResponse extends ClusterManagementWorkResponse {
+
+    private final List<String> nodesToBeAdded;
+    private final List<String> nodesAdded;
+
+    public AddNodeWorkResponse(AddNodeWork w, List<String> nodesToBeAdded) {
+        super(w);
+        this.nodesToBeAdded = nodesToBeAdded;
+        this.nodesAdded = new ArrayList<String>();
+    }
+
+    public List<String> getNodesAdded() {
+        return nodesAdded;
+    }
+
+    public boolean updateProgress(String addedNode) {
+        nodesToBeAdded.remove(addedNode);
+        nodesAdded.add(addedNode);
+        return nodesToBeAdded.isEmpty();
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java
new file mode 100644
index 0000000..d578a77
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public class ClusterManagementWorkResponse implements IClusterManagementWorkResponse {
+
+    protected final IClusterManagementWork work;
+
+    protected Status status;
+
+    public ClusterManagementWorkResponse(IClusterManagementWork w) {
+        this.work = w;
+        this.status = Status.IN_PROGRESS;
+    }
+
+    @Override
+    public IClusterManagementWork getWork() {
+        return work;
+    }
+
+    @Override
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(Status status) {
+        this.status = status;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java
new file mode 100644
index 0000000..cefb431
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java
@@ -0,0 +1,169 @@
+/*
+ * 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.cluster;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManager;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+public class ClusterManager implements IClusterManager {
+
+    private static final Logger LOGGER = Logger.getLogger(AsterixEventServiceClient.class.getName());
+
+    public static ClusterManager INSTANCE = new ClusterManager();
+
+    private static String eventsDir = System.getenv("user.dir") + File.separator + "eventrix";
+
+    private static AsterixEventServiceClient client;
+
+    private static ILookupService lookupService;
+
+    private static final Set<IClusterEventsSubscriber> eventSubscribers = new HashSet<IClusterEventsSubscriber>();
+
+    private ClusterManager() {
+        Cluster asterixCluster = AsterixClusterProperties.INSTANCE.getCluster();
+        String eventHome = asterixCluster == null ? null : asterixCluster.getWorkingDir().getDir();
+
+        if (asterixCluster != null) {
+            String asterixDir = System.getProperty("user.dir") + File.separator + "asterix";
+            File configFile = new File(System.getProperty("user.dir") + File.separator + "configuration.xml");
+            Configuration configuration = null;
+
+            try {
+                JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
+                Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+                configuration = (Configuration) unmarshaller.unmarshal(configFile);
+                AsterixEventService.initialize(configuration, asterixDir, eventHome);
+                client = AsterixEventService.getAsterixEventServiceClient(AsterixClusterProperties.INSTANCE
+                        .getCluster());
+
+                lookupService = ServiceProvider.INSTANCE.getLookupService();
+                if (!lookupService.isRunning(configuration)) {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Lookup service not running. Starting lookup service ...");
+                    }
+                    lookupService.startService(configuration);
+                } else {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Lookup service running");
+                    }
+                }
+
+            } catch (Exception e) {
+                throw new IllegalStateException("Unable to initialize cluster manager" + e);
+            }
+        }
+    }
+
+    @Override
+    public void addNode(Node node) throws AsterixException {
+        try {
+            Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+            List<Pattern> pattern = new ArrayList<Pattern>();
+            String asterixInstanceName = AsterixAppContextInfo.getInstance().getMetadataProperties().getInstanceName();
+            Patterns prepareNode = PatternCreator.INSTANCE.createPrepareNodePattern(asterixInstanceName,
+                    AsterixClusterProperties.INSTANCE.getCluster(), node);
+            cluster.getNode().add(node);
+            client.submit(prepareNode);
+
+            AsterixExternalProperties externalProps = AsterixAppContextInfo.getInstance().getExternalProperties();
+            AsterixEventServiceUtil.poulateClusterEnvironmentProperties(cluster, externalProps.getCCJavaParams(),
+                    externalProps.getNCJavaParams());
+
+            pattern.clear();
+            String ccHost = cluster.getMasterNode().getClusterIp();
+            String hostId = node.getId();
+            String nodeControllerId = asterixInstanceName + "_" + node.getId();
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            Pattern startNC = PatternCreator.INSTANCE.createNCStartPattern(ccHost, hostId, nodeControllerId, iodevices);
+            pattern.add(startNC);
+            Patterns startNCPattern = new Patterns(pattern);
+            client.submit(startNCPattern);
+
+            removeNode(cluster.getSubstituteNodes().getNode(), node);
+
+            AsterixInstance instance = lookupService.getAsterixInstance(cluster.getInstanceName());
+            instance.getCluster().getNode().add(node);
+            removeNode(instance.getCluster().getSubstituteNodes().getNode(), node);
+            lookupService.updateAsterixInstance(instance);
+
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+
+    }
+
+    private void removeNode(List<Node> list, Node node) {
+        Node nodeToRemove = null;
+        for (Node n : list) {
+            if (n.getId().equals(node.getId())) {
+                nodeToRemove = n;
+                break;
+            }
+        }
+        if (nodeToRemove != null) {
+            boolean removed = list.remove(nodeToRemove);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("attempt to remove node :" + nodeToRemove + " successful " + removed);
+            }
+        }
+    }
+
+    @Override
+    public void removeNode(Node node) throws AsterixException {
+        // to be implemented later.
+    }
+
+    @Override
+    public void registerSubscriber(IClusterEventsSubscriber subscriber) {
+        eventSubscribers.add(subscriber);
+    }
+
+    @Override
+    public boolean deregisterSubscriber(IClusterEventsSubscriber subscriber) {
+        return eventSubscribers.remove(subscriber);
+    }
+
+    @Override
+    public Set<IClusterEventsSubscriber> getRegisteredClusterEventSubscribers() {
+        return eventSubscribers;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
new file mode 100644
index 0000000..dfc88ac
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public interface IClusterManagementWorkResponse {
+
+    public enum Status {
+        IN_PROGRESS,
+        SUCCESS,
+        FAILURE
+    }
+
+    /**
+     * @return
+     */
+    public IClusterManagementWork getWork();
+
+    /**
+     * @return
+     */
+    public Status getStatus();
+
+    /**
+     * @param status
+     */
+    public void setStatus(Status status);
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java
new file mode 100644
index 0000000..90683d1
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.util.Set;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+
+public class RemoveNodeWork extends AbstractClusterManagementWork {
+
+    private final Set<String> nodesToBeRemoved;
+
+    @Override
+    public WorkType getClusterManagementWorkType() {
+        return WorkType.REMOVE_NODE;
+    }
+
+    public RemoveNodeWork(Set<String> nodesToBeRemoved, IClusterEventsSubscriber subscriber) {
+        super(subscriber);
+        this.nodesToBeRemoved = nodesToBeRemoved;
+    }
+
+    public Set<String> getNodesToBeRemoved() {
+        return nodesToBeRemoved;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append(WorkType.REMOVE_NODE);
+        for (String node : nodesToBeRemoved) {
+            builder.append(node + " ");
+        }
+        builder.append(" requested by " + subscriber);
+        return builder.toString();
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java
new file mode 100644
index 0000000..58ea05e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java
@@ -0,0 +1,34 @@
+/*
+ * 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.cluster;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class RemoveNodeWorkResponse extends ClusterManagementWorkResponse {
+
+    private Set<String> nodesToBeRemoved = new HashSet<String>();
+
+    public RemoveNodeWorkResponse(RemoveNodeWork w, Status status) {
+        super(w);
+        nodesToBeRemoved.addAll(w.getNodesToBeRemoved());
+    }
+
+    public boolean updateProgress(Set<String> failedNodeIds) {
+        nodesToBeRemoved.removeAll(failedNodeIds);
+        return nodesToBeRemoved.isEmpty();
+
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
index 0b7a7d9..2a80963 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
@@ -17,6 +17,8 @@
 import java.util.HashSet;
 import java.util.Set;
 
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
 /**
@@ -51,6 +53,7 @@
     private static Set<IHint> initHints() {
         Set<IHint> hints = new HashSet<IHint>();
         hints.add(new DatasetCardinalityHint());
+        hints.add(new DatasetNodegroupCardinalityHint());
         return hints;
     }
 
@@ -84,4 +87,43 @@
         }
 
     }
+
+    /**
+     * Hint representing the cardinality of nodes forming the nodegroup for the dataset.
+     */
+    public static class DatasetNodegroupCardinalityHint implements IHint {
+        public static final String NAME = "NODEGROUP_CARDINALITY";
+
+        public static final int DEFAULT = 1;
+
+        @Override
+        public String getName() {
+            return NAME;
+        }
+
+        @Override
+        public Pair<Boolean, String> validateValue(String value) {
+            boolean valid = true;
+            int intValue;
+            try {
+                intValue = Integer.parseInt(value);
+                if (intValue < 0) {
+                    return new Pair<Boolean, String>(false, "Value must be >= 0");
+                }
+                int numNodesInCluster = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames()
+                        .size();
+                if (numNodesInCluster < intValue) {
+                    return new Pair<Boolean, String>(false,
+                            "Value must be greater or equal to the existing number of nodes in cluster ("
+                                    + numNodesInCluster + ")");
+                }
+            } catch (NumberFormatException nfe) {
+                valid = false;
+                return new Pair<Boolean, String>(valid, "Inappropriate value");
+            }
+            return new Pair<Boolean, String>(true, null);
+        }
+
+    }
+
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
index d4a937d..519c831 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
+
 /**
  * Represents a hint provided as part of an AQL statement.
  */
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 166de00..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,16 +15,13 @@
 
 package edu.uci.ics.asterix.metadata.declared;
 
-import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+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;
@@ -32,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.DefaultNodeGroupDomain;
 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;
@@ -44,142 +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:
-                    initFeedDataset(itemType, dataset);
-                    break;
-                case INTERNAL:
-                    initInternalDataset(itemType);
-                    break;
-                case 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 DefaultNodeGroupDomain(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 DefaultNodeGroupDomain(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
@@ -198,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;
@@ -231,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) {
@@ -254,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;
         }
 
     }
@@ -268,4 +185,12 @@
         return datasourceType;
     }
 
+    public Map<String, Serializable> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(Map<String, Serializable> properties) {
+        this.properties = properties;
+    }
+
 }
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 8301f5d..46b8268 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
@@ -40,17 +40,6 @@
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataScanOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.FeedIntakeOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.FeedMessageOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -59,14 +48,30 @@
 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.FeedDatasetDetails;
+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.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.EndFeedMessage;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
+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;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -110,7 +115,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
-import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -129,7 +134,6 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -161,14 +165,14 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+    private IHyracksClientConnection hcc;
 
     private final Dataverse defaultDataverse;
     private JobId jobId;
 
     private final AsterixStorageProperties storageProperties;
 
-    private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
-    private static Scheduler hdfsScheduler;
+    public static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
 
     public String getPropertyValue(String propertyName) {
         return config.get(propertyName);
@@ -190,16 +194,6 @@
         this.defaultDataverse = defaultDataverse;
         this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
         this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
-        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
-        try {
-            if (hdfsScheduler == null) {
-                //set the singleton hdfs scheduler
-                hdfsScheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
-                        .getClusterControllerInfo().getClientNetPort());
-            }
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
     }
 
     public void setJobId(JobId jobId) {
@@ -286,34 +280,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);
@@ -322,27 +302,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();
-        if (dataSource instanceof ExternalFeedDataSource) {
-            return buildFeedIntakeRuntime(jobSpec, dataset);
-        } else {
-            return buildExternalDataScannerRuntime(jobSpec, itemType,
-                    (ExternalDatasetDetails) dataset.getDatasetDetails(), NonTaggedDataFormat.INSTANCE);
-        }
+        return buildExternalDataScannerRuntime(jobSpec, itemType, (ExternalDatasetDetails) dataset.getDatasetDetails(),
+                NonTaggedDataFormat.INSTANCE);
+
     }
 
     @SuppressWarnings("rawtypes")
@@ -353,8 +332,7 @@
             throw new AlgebricksException("Can only scan datasets of records.");
         }
 
-        IGenericDatasetAdapterFactory adapterFactory;
-        IDatasourceAdapter adapter;
+        IAdapterFactory adapterFactory;
         String adapterName;
         DatasourceAdapter adapterEntity;
         String adapterFactoryClassname;
@@ -364,17 +342,25 @@
                     adapterName);
             if (adapterEntity != null) {
                 adapterFactoryClassname = adapterEntity.getClassname();
-                adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
             } else {
                 adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
                 if (adapterFactoryClassname == null) {
                     throw new AlgebricksException(" Unknown adapter :" + adapterName);
                 }
-                adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
             }
 
-            adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
-                    wrapProperties(datasetDetails.getProperties()), itemType);
+            Map<String, String> configuration = datasetDetails.getProperties();
+
+            switch (adapterFactory.getAdapterType()) {
+                case GENERIC:
+                    ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType);
+                    break;
+                case TYPED:
+                    ((ITypedAdapterFactory) adapterFactory).configure(configuration);
+                    break;
+            }
         } catch (AlgebricksException ae) {
             throw ae;
         } catch (Exception e) {
@@ -382,21 +368,20 @@
             throw new AlgebricksException("Unable to create adapter " + e);
         }
 
-        if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
-                IDatasourceAdapter.AdapterType.READ_WRITE))) {
+        if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ) || adapterFactory
+                .getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
             throw new AlgebricksException("external dataset adapter does not support read operation");
         }
-        ARecordType rt = (ARecordType) itemType;
 
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
-        ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
-                wrapPropertiesEmpty(datasetDetails.getProperties()), rt, scannerDesc, adapterFactory);
+        ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc,
+                adapterFactory);
 
         AlgebricksPartitionConstraint constraint;
         try {
-            constraint = adapter.getPartitionConstraint();
+            constraint = adapterFactory.getPartitionConstraint();
         } catch (Exception e) {
             throw new AlgebricksException(e);
         }
@@ -428,80 +413,72 @@
 
     @SuppressWarnings("rawtypes")
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(JobSpecification jobSpec,
-            Dataset dataset) throws AlgebricksException {
+            IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
 
-        FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
-        DatasourceAdapter adapterEntity;
-        IDatasourceAdapter adapter;
-        IAdapterFactory adapterFactory;
-        IAType adapterOutputType;
-        String adapterName;
-        String adapterFactoryClassname;
-
-        try {
-            adapterName = datasetDetails.getAdapterFactory();
-            adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
-                    adapterName);
-            if (adapterEntity != null) {
-                adapterFactoryClassname = adapterEntity.getClassname();
-                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-            } else {
-                adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
-                if (adapterFactoryClassname != null) {
-                } else {
-                    // adapterName has been provided as a fully qualified
-                    // classname
-                    adapterFactoryClassname = adapterName;
-                }
-                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-            }
-
-            if (adapterFactory instanceof ITypedDatasetAdapterFactory) {
-                adapter = ((ITypedDatasetAdapterFactory) adapterFactory).createAdapter(wrapProperties(datasetDetails
-                        .getProperties()));
-                adapterOutputType = ((ITypedDatasourceAdapter) adapter).getAdapterOutputType();
-            } else if (adapterFactory instanceof IGenericDatasetAdapterFactory) {
-                String outputTypeName = datasetDetails.getProperties().get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME);
-                adapterOutputType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(),
-                        outputTypeName).getDatatype();
-                adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
-                        wrapProperties(datasetDetails.getProperties()), adapterOutputType);
-            } else {
-                throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
-            }
-        } catch (AlgebricksException ae) {
-            throw ae;
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new AlgebricksException("unable to create adapter  " + e);
-        }
-
-        ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
-                .getSerializerDeserializer(adapterOutputType);
-        RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-
-        FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(
-                dataset.getDataverseName(), dataset.getDatasetName()), adapterFactoryClassname,
-                this.wrapPropertiesEmpty(datasetDetails.getProperties()), (ARecordType) adapterOutputType, feedDesc,
-                adapterFactory);
-
+        FeedDataSource feedDataSource = (FeedDataSource) dataSource;
+        FeedIntakeOperatorDescriptor feedIngestor = null;
+        org.apache.commons.lang3.tuple.Pair<IAdapterFactory, ARecordType> factoryOutput = null;
         AlgebricksPartitionConstraint constraint = null;
+
         try {
-            constraint = adapter.getPartitionConstraint();
+            factoryOutput = FeedUtil.getFeedFactoryAndOutput(feedDataSource.getFeed(), mdTxnCtx);
+            IAdapterFactory adapterFactory = factoryOutput.getLeft();
+            ARecordType adapterOutputType = factoryOutput.getRight();
+
+            ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
+                    .getSerializerDeserializer(adapterOutputType);
+            RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+
+            FeedPolicy feedPolicy = (FeedPolicy) ((AqlDataSource) dataSource).getProperties().get(
+                    BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+
+            feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
+            feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedConnectionId(
+                    feedDataSource.getDatasourceDataverse(), feedDataSource.getDatasourceName(), feedDataSource
+                            .getFeedConnectionId().getDatasetName()), adapterFactory, (ARecordType) adapterOutputType,
+                    feedDesc, feedPolicy.getProperties());
+
+            constraint = factoryOutput.getLeft().getPartitionConstraint();
         } catch (Exception e) {
             throw new AlgebricksException(e);
         }
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
     }
 
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
-            AqlMetadataProvider metadataProvider, JobSpecification jobSpec, FeedDatasetDetails datasetDetails,
-            String dataverse, String dataset, List<IFeedMessage> feedMessages) throws AlgebricksException {
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadataProvider
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverse, dataset, dataset);
-        FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
-                feedMessages);
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, spPc.second);
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
+            JobSpecification jobSpec, String dataverse, String feedName, String dataset, IFeedMessage feedMessage,
+            String[] locations) throws AlgebricksException {
+        AlgebricksPartitionConstraint partitionConstraint = new AlgebricksAbsolutePartitionConstraint(locations);
+        FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, feedName,
+                dataset, feedMessage);
+        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
+    }
+
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
+            JobSpecification jobSpec, String dataverse, String feedName, String dataset, FeedActivity feedActivity)
+            throws AlgebricksException {
+        List<String> feedLocations = new ArrayList<String>();
+        String[] ingestLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.INGEST_LOCATIONS)
+                .split(",");
+        String[] computeLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.COMPUTE_LOCATIONS)
+                .split(",");
+        String[] storageLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.STORAGE_LOCATIONS)
+                .split(",");
+
+        for (String loc : ingestLocs) {
+            feedLocations.add(loc);
+        }
+        for (String loc : computeLocs) {
+            feedLocations.add(loc);
+        }
+        for (String loc : storageLocs) {
+            feedLocations.add(loc);
+        }
+
+        FeedConnectionId feedId = new FeedConnectionId(dataverse, feedName, dataset);
+        String[] locations = feedLocations.toArray(new String[] {});
+        IFeedMessage feedMessage = new EndFeedMessage(feedId);
+        return buildSendFeedMessageRuntime(jobSpec, dataverse, feedName, dataset, feedMessage, locations);
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
@@ -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
@@ -1419,8 +1396,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())
@@ -1516,6 +1493,22 @@
         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);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
         try {
             return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -1554,8 +1547,8 @@
     private Map<String, Object> wrapProperties(Map<String, String> properties) {
         Map<String, Object> wrappedProperties = new HashMap<String, Object>();
         wrappedProperties.putAll(properties);
-        wrappedProperties.put(HDFSAdapterFactory.SCHEDULER, hdfsScheduler);
-        wrappedProperties.put(HDFSAdapterFactory.CLUSTER_LOCATIONS, getClusterLocations());
+        //wrappedProperties.put(SCHEDULER, hdfsScheduler);
+        //wrappedProperties.put(CLUSTER_LOCATIONS, getClusterLocations());
         return wrappedProperties;
     }
 
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..cb1124f
--- /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.DefaultNodeGroupDomain;
+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 DefaultNodeGroupDomain(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/declared/FileSplitDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
index 3e61ec5..0827f09 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
@@ -18,6 +18,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 
+
 public class FileSplitDataSink implements IDataSink {
 
     private FileSplitSinkId id;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
index 5801002..9ca7291 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
@@ -17,6 +17,7 @@
 
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 
+
 public class FileSplitSinkId {
 
     private FileSplit fileSplit;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
index 3fd9285..50c6225 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ResultSetDomain;
 
+
 public class ResultSetDataSink implements IDataSink {
 
     private ResultSetSinkId id;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
index 99d026b..4fa3907 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 
+
 public class ResultSetSinkId {
 
     private final ResultSetId resultSetId;
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/DatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
index 3c72084..621933f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
@@ -14,12 +14,14 @@
  */
 package edu.uci.ics.asterix.metadata.entities;
 
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.MetadataCache;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
 
 public class DatasourceAdapter implements IMetadataEntity {
 
+    private static final long serialVersionUID = 1L;
+
     public enum AdapterType {
         INTERNAL,
         EXTERNAL
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
new file mode 100644
index 0000000..679276f
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
@@ -0,0 +1,175 @@
+/*
+ * Copyright 2009-2010 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.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed activity record.
+ */
+public class FeedActivity implements IMetadataEntity, Comparable<FeedActivity> {
+
+    private static final long serialVersionUID = 1L;
+
+    private int activityId;
+
+    private final String dataverseName;
+    private final String datasetName;
+    private final String feedName;
+
+    private String lastUpdatedTimestamp;
+    private FeedActivityType activityType;
+    private Map<String, String> feedActivityDetails;
+
+    public static enum FeedActivityType {
+        FEED_BEGIN,
+        FEED_FAILURE,
+        FEED_END
+    }
+
+    public static class FeedActivityDetails {
+        public static final String COMPUTE_LOCATIONS = "compute-locations";
+        public static final String INGEST_LOCATIONS = "ingest-locations";
+        public static final String STORAGE_LOCATIONS = "storage-locations";
+        public static final String TOTAL_INGESTED = "total-ingested";
+        public static final String INGESTION_RATE = "ingestion-rate";
+        public static final String EXCEPTION_LOCATION = "exception-location";
+        public static final String EXCEPTION_MESSAGE = "exception-message";
+        public static final String FEED_POLICY_NAME = "feed-policy-name";
+        public static final String SUPER_FEED_MANAGER_HOST = "super-feed-manager-host";
+        public static final String SUPER_FEED_MANAGER_PORT = "super-feed-manager-port";
+        public static final String FEED_NODE_FAILURE = "feed-node-failure";
+
+    }
+
+    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;
+    }
+
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public String getFeedName() {
+        return feedName;
+    }
+
+    @Override
+    public Object addToCache(MetadataCache cache) {
+        return cache.addFeedActivityIfNotExists(this);
+    }
+
+    @Override
+    public Object dropFromCache(MetadataCache cache) {
+        return cache.dropFeedActivity(this);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof FeedActivity)) {
+            return false;
+        }
+
+        if (!((FeedActivity) other).dataverseName.equals(dataverseName)) {
+            return false;
+        }
+        if (!((FeedActivity) other).datasetName.equals(datasetName)) {
+            return false;
+        }
+        if (!((FeedActivity) other).getFeedName().equals(feedName)) {
+            return false;
+        }
+        if (!((FeedActivity) other).getFeedActivityType().equals(activityType)) {
+            return false;
+        }
+        if (((FeedActivity) other).getActivityId() != (activityId)) {
+            return false;
+        }
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        return toString().hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return dataverseName + "." + feedName + " --> " + datasetName + " " + activityType + " " + activityId;
+    }
+
+    public FeedActivityType getFeedActivityType() {
+        return activityType;
+    }
+
+    public void setFeedActivityType(FeedActivityType feedActivityType) {
+        this.activityType = feedActivityType;
+    }
+
+    public String getLastUpdatedTimestamp() {
+        return lastUpdatedTimestamp;
+    }
+
+    public void setLastUpdatedTimestamp(String lastUpdatedTimestamp) {
+        this.lastUpdatedTimestamp = lastUpdatedTimestamp;
+    }
+
+    public int getActivityId() {
+        return activityId;
+    }
+
+    public void setActivityId(int activityId) {
+        this.activityId = activityId;
+    }
+
+    public Map<String, String> getFeedActivityDetails() {
+        return feedActivityDetails;
+    }
+
+    public void setFeedActivityDetails(Map<String, String> feedActivityDetails) {
+        this.feedActivityDetails = feedActivityDetails;
+    }
+
+    public FeedActivityType getActivityType() {
+        return activityType;
+    }
+
+    public void setActivityType(FeedActivityType activityType) {
+        this.activityType = activityType;
+    }
+
+    @Override
+    public int compareTo(FeedActivity o) {
+        return o.getActivityId() - this.activityId;
+    }
+
+}
\ No newline at end of file
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 5058e24..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ /dev/null
@@ -1,215 +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;
-    private FeedState feedState;
-
-    public enum FeedState {
-        INACTIVE,
-        // INACTIVE state signifies that the feed dataset is not
-        // connected with the external world through the feed
-        // adapter.
-        ACTIVE
-        // ACTIVE state signifies that the feed dataset is connected to the
-        // external world using an adapter that may put data into the dataset.
-    }
-
-    public FeedDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
-            List<String> partitioningKey, List<String> primaryKey, String groupName, String adapterFactory,
-            Map<String, String> properties, FunctionSignature signature, String feedState, String compactionPolicy,
-            Map<String, String> compactionPolicyProperties) {
-        super(fileStructure, partitioningStrategy, partitioningKey, primaryKey, groupName, compactionPolicy,
-                compactionPolicyProperties);
-        this.properties = properties;
-        this.adapterFactory = adapterFactory;
-        this.signature = signature;
-        this.feedState = feedState.equals(FeedState.ACTIVE.toString()) ? FeedState.ACTIVE : FeedState.INACTIVE;
-    }
-
-    @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(),
-                    MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
-            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);
-        }
-
-        // write field 8
-        fieldValue.reset();
-        aString.setValue(getFeedState().toString());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX, fieldValue);
-
-        // write field 9
-        fieldValue.reset();
-        aString.setValue(getCompactionPolicy().toString());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX, fieldValue);
-
-        // write field 10
-        listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[10]);
-        for (Map.Entry<String, String> property : compactionPolicyProperties.entrySet()) {
-            String name = property.getKey();
-            String value = property.getValue();
-            itemValue.reset();
-            writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
-                    MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
-            listBuilder.addItem(itemValue);
-        }
-        fieldValue.reset();
-        listBuilder.write(fieldValue.getDataOutput(), true);
-        feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX,
-                fieldValue);
-
-        try {
-            feedRecordBuilder.write(out, true);
-        } catch (IOException | AsterixException e) {
-            throw new HyracksDataException(e);
-        }
-
-    }
-
-    public FeedState getFeedState() {
-        return feedState;
-    }
-
-    public void setFeedState(FeedState feedState) {
-        this.feedState = feedState;
-    }
-
-    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/entities/FeedPolicy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedPolicy.java
new file mode 100644
index 0000000..b011e5c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedPolicy.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 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.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed activity record.
+ */
+public class FeedPolicy implements IMetadataEntity {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String dataverseName;
+    // Enforced to be unique within a dataverse.
+    private final String policyName;
+    // A description of the policy
+    private final String description;
+    // The policy properties associated with the feed dataset
+    private Map<String, String> properties;
+
+    public FeedPolicy(String dataverseName, String policyName, String description, Map<String, String> properties) {
+        this.dataverseName = dataverseName;
+        this.policyName = policyName;
+        this.description = description;
+        this.properties = properties;
+    }
+
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getPolicyName() {
+        return policyName;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof FeedPolicy)) {
+            return false;
+        }
+        FeedPolicy otherPolicy = (FeedPolicy) other;
+        if (!otherPolicy.dataverseName.equals(dataverseName)) {
+            return false;
+        }
+        if (!otherPolicy.policyName.equals(policyName)) {
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public Object addToCache(MetadataCache cache) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Object dropFromCache(MetadataCache cache) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(Map<String, String> properties) {
+        this.properties = properties;
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java
new file mode 100644
index 0000000..a341ba9
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.metadata.entities;
+
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+public class Library implements IMetadataEntity {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String dataverse;
+    private final String name;
+
+    public Library(String dataverseName, String libraryName) {
+        this.dataverse = dataverseName;
+        this.name = libraryName;
+    }
+
+    public String getDataverseName() {
+        return dataverse;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public Object addToCache(MetadataCache cache) {
+        return cache.addLibraryIfNotExists(this);
+    }
+
+    @Override
+    public Object dropFromCache(MetadataCache cache) {
+        return cache.dropLibrary(this);
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
index dc8c390..a85787f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntityTupleTranslator;
 import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableString;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -40,6 +41,10 @@
     @SuppressWarnings("unchecked")
     protected ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+
     protected final IARecordBuilder recordBuilder;
     protected final ArrayBackedValueStorage fieldValue;
     protected final ArrayTupleBuilder tupleBuilder;
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 3f70e2b..2bf666c 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,86 +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]));
-                }
-
-                String feedState = ((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX)).getStringValue();
-
-                String compactionPolicy = ((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX))
-                        .getStringValue();
-                cursor = ((AOrderedList) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX))
-                        .getCursor();
-                Map<String, String> compactionPolicyProperties = 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();
-                    compactionPolicyProperties.put(key, value);
-                }
-
-                datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
-                        partitioningKey, groupName, adapter, properties, signature, feedState, compactionPolicy,
-                        compactionPolicyProperties);
-                break;
-            }
             case INTERNAL: {
                 ARecord datasetDetailsRecord = (ARecord) datasetRecord
                         .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX);
@@ -363,9 +281,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;
         }
 
     }
@@ -379,10 +294,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/DatasourceAdapterTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index 4a5e4dcf..e93fd97 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -22,13 +22,13 @@
 import java.util.Calendar;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 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.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
 import edu.uci.ics.asterix.om.base.ARecord;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
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
new file mode 100644
index 0000000..bcc16a7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
@@ -0,0 +1,244 @@
+/*
+ * 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.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.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+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.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 Dataset metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedActivityTupleTranslator extends AbstractTupleTranslator<FeedActivity> {
+    // Field indexes of serialized FeedActivity in a tuple.
+    // Key field.
+    public static final int FEED_ACTIVITY_ACTIVITY_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+    public static final int FEED_ACTIVITY_ACTIVITY_FEED_NAME_FIELD_INDEX = 1;
+
+    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 = 4;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE);
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+    @SuppressWarnings("unchecked")
+    public FeedActivityTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1);
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+    }
+
+    @Override
+    public FeedActivity getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+        byte[] serRecord = frameTuple.getFieldData(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordStartOffset = frameTuple.getFieldStart(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = frameTuple.getFieldLength(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord feedActivityRecord = (ARecord) recordSerDes.deserialize(in);
+        return createFeedActivityFromARecord(feedActivityRecord);
+    }
+
+    private FeedActivity createFeedActivityFromARecord(ARecord feedActivityRecord) {
+
+        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
+                .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX)).getIntegerValue();
+        String feedActivityType = ((AString) feedActivityRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX)).getStringValue();
+
+        IACursor cursor = ((AUnorderedList) feedActivityRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX)).getCursor();
+        Map<String, String> activityDetails = 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();
+            activityDetails.put(key, value);
+        }
+
+        String feedActivityTimestamp = ((AString) feedActivityRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX))
+                .getStringValue();
+
+        FeedActivity fa = new FeedActivity(dataverseName, feedName, datasetName,
+                FeedActivityType.valueOf(feedActivityType), activityDetails);
+        fa.setLastUpdatedTimestamp(feedActivityTimestamp);
+        fa.setActivityId(activityId);
+        return fa;
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(FeedActivity feedActivity) throws IOException, MetadataException {
+        // write the key in the first three fields of the tuple
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+        tupleBuilder.reset();
+        aString.setValue(feedActivity.getDataverseName());
+        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();
+
+        aInt32.setValue(feedActivity.getActivityId());
+        int32Serde.serialize(aInt32, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        // write the pay-load in the 2nd field of the tuple
+
+        recordBuilder.reset(MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(feedActivity.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        // 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 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 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 5
+        Map<String, String> properties = feedActivity.getFeedActivityDetails();
+        UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+        listBuilder
+                .reset((AUnorderedListType) MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX]);
+        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);
+        recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX, fieldValue);
+
+        // write field 6
+        fieldValue.reset();
+        aString.setValue(Calendar.getInstance().getTime().toString());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_LAST_UPDATE_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_ACTIVITY_DETAILS_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);
+        }
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
new file mode 100644
index 0000000..57d2d1c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -0,0 +1,200 @@
+/*
+ * 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.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.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.FeedPolicy;
+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.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 Dataset metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedPolicyTupleTranslator extends AbstractTupleTranslator<FeedPolicy> {
+    // Field indexes of serialized FeedPolicy in a tuple.
+    // Key field.
+    public static final int FEED_POLICY_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+    public static final int FEED_POLICY_POLICY_NAME_FIELD_INDEX = 1;
+
+    // Payload field containing serialized feedPolicy.
+    public static final int FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.FEED_POLICY_RECORDTYPE);
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+    @SuppressWarnings("unchecked")
+    public FeedPolicyTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FEED_POLICY_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1);
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+    }
+
+    @Override
+    public FeedPolicy getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+        byte[] serRecord = frameTuple.getFieldData(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordStartOffset = frameTuple.getFieldStart(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = frameTuple.getFieldLength(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord feedPolicyRecord = (ARecord) recordSerDes.deserialize(in);
+        return createFeedPolicyFromARecord(feedPolicyRecord);
+    }
+
+    private FeedPolicy createFeedPolicyFromARecord(ARecord feedPolicyRecord) {
+        FeedPolicy feedPolicy = null;
+        String dataverseName = ((AString) feedPolicyRecord
+                .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+        String policyName = ((AString) feedPolicyRecord
+                .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX)).getStringValue();
+
+        String description = ((AString) feedPolicyRecord
+                .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_DESCRIPTION_FIELD_INDEX)).getStringValue();
+
+        IACursor cursor = ((AUnorderedList) feedPolicyRecord
+                .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX)).getCursor();
+        Map<String, String> policyParamters = 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();
+            policyParamters.put(key, value);
+        }
+
+        feedPolicy = new FeedPolicy(dataverseName, policyName, description, policyParamters);
+        return feedPolicy;
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(FeedPolicy feedPolicy) throws IOException, MetadataException {
+        // write the key in the first three fields of the tuple
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+        tupleBuilder.reset();
+        aString.setValue(feedPolicy.getDataverseName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        aString.setValue(feedPolicy.getPolicyName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        recordBuilder.reset(MetadataRecordTypes.FEED_POLICY_RECORDTYPE);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(feedPolicy.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(feedPolicy.getPolicyName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 2
+        fieldValue.reset();
+        aString.setValue(feedPolicy.getDescription());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 3 (properties)
+        Map<String, String> properties = feedPolicy.getProperties();
+        UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+        listBuilder
+                .reset((AUnorderedListType) MetadataRecordTypes.FEED_POLICY_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX]);
+        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);
+        recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_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.POLICY_PARAMS_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/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/entitytupletranslators/LibraryTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
new file mode 100644
index 0000000..4bd2366
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -0,0 +1,120 @@
+/*
+ * 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.IOException;
+import java.util.Calendar;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+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.Library;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Library metadata entity to an ITupleReference and vice versa.
+ */
+public class LibraryTupleTranslator extends AbstractTupleTranslator<Library> {
+    // Field indexes of serialized Library in a tuple.
+    // First key field.
+    public static final int LIBRARY_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
+    // Second key field.
+    public static final int LIBRARY_NAME_TUPLE_FIELD_INDEX = 1;
+
+    // Payload field containing serialized Library.
+    public static final int LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.LIBRARY_RECORDTYPE);
+
+    public LibraryTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.LIBRARY_DATASET.getFieldCount());
+    }
+
+    @Override
+    public Library getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+        byte[] serRecord = frameTuple.getFieldData(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordStartOffset = frameTuple.getFieldStart(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = frameTuple.getFieldLength(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord libraryRecord = (ARecord) recordSerDes.deserialize(in);
+        return createLibraryFromARecord(libraryRecord);
+    }
+
+    private Library createLibraryFromARecord(ARecord libraryRecord) {
+        String dataverseName = ((AString) libraryRecord
+                .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
+        String libraryName = ((AString) libraryRecord
+                .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX)).getStringValue();
+
+        return new Library(dataverseName, libraryName);
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(Library library) throws IOException, MetadataException {
+        // write the key in the first 2 fields of the tuple
+        tupleBuilder.reset();
+        aString.setValue(library.getDataverseName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        aString.setValue(library.getName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        // write the pay-load in the third field of the tuple
+
+        recordBuilder.reset(MetadataRecordTypes.LIBRARY_RECORDTYPE);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(library.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(library.getName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 2
+        fieldValue.reset();
+        aString.setValue(Calendar.getInstance().getTime().toString());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.LIBRARY_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;
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java
new file mode 100644
index 0000000..9e8e5f7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java
@@ -0,0 +1,51 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Represents the base class that is required to be extended by every
+ * implementation of the IDatasourceAdapter interface.
+ */
+public abstract class AbstractDatasourceAdapter implements IDatasourceAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    protected Map<String, Object> configuration;
+    protected transient AlgebricksPartitionConstraint partitionConstraint;
+    protected IAType atype;
+    protected IHyracksTaskContext ctx;
+
+    protected static final Map<String, Object> formatToParserFactoryMap = initializeFormatParserFactoryMap();
+
+    public static final String KEY_FORMAT = "format";
+    public static final String KEY_PARSER_FACTORY = "parser";
+    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String FORMAT_ADM = "adm";
+
+    private static Map<String, Object> initializeFormatParserFactoryMap() {
+        Map<String, Object> map = new HashMap<String, Object>();
+        map.put(FORMAT_DELIMITED_TEXT, "edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory");
+        map.put(FORMAT_ADM, "edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory");
+        return map;
+    }
+
+}
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
new file mode 100644
index 0000000..4921920
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+
+public abstract class AbstractFeedDatasourceAdapter implements IDatasourceAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    protected FeedPolicyEnforcer policyEnforcer;
+
+    public FeedPolicyEnforcer getPolicyEnforcer() {
+        return policyEnforcer;
+    }
+
+    public void setFeedPolicyEnforcer(FeedPolicyEnforcer policyEnforcer) {
+        this.policyEnforcer = policyEnforcer;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
similarity index 96%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
index f6164ea..897faae 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.dataset.adapter;
+package edu.uci.ics.asterix.metadata.feeds;
 
 import java.io.Serializable;
 
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
new file mode 100644
index 0000000..44ad1e9
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
@@ -0,0 +1,200 @@
+/*
+ * 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.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedFrameWriter.Mode;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class AdapterRuntimeManager implements IAdapterExecutor {
+
+    private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
+
+    private final FeedConnectionId feedId;
+
+    private IFeedAdapter feedAdapter;
+
+    private AdapterExecutor adapterExecutor;
+
+    private State state;
+
+    private int partition;
+
+    private IngestionRuntime ingestionRuntime;
+
+    public enum State {
+        /*
+         * Indicates that data from external source will be pushed downstream for storage 
+         */
+        ACTIVE_INGESTION,
+        /*
+         * Indicates that data from external source would be buffered and not pushed downstream 
+         */
+        INACTIVE_INGESTION,
+        /*
+         * Indicates that feed ingestion activity has finished
+         */
+        FINISHED_INGESTION
+    }
+
+    public AdapterRuntimeManager(FeedConnectionId feedId, IFeedAdapter feedAdapter, FeedFrameWriter writer,
+            int partition, LinkedBlockingQueue<IFeedMessage> inbox) {
+        this.feedId = feedId;
+        this.feedAdapter = feedAdapter;
+        this.partition = partition;
+        this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
+    }
+
+    @Override
+    public void start() throws Exception {
+        state = State.ACTIVE_INGESTION;
+        ingestionRuntime = new IngestionRuntime(feedId, partition, FeedRuntimeType.INGESTION, this);
+        FeedManager.INSTANCE.registerFeedRuntime(ingestionRuntime);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Registered feed runtime manager for " + this.getFeedId());
+        }
+        ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedId);
+        executorService.execute(adapterExecutor);
+    }
+
+    @Override
+    public void stop() {
+        try {
+            feedAdapter.stop();
+            state = State.FINISHED_INGESTION;
+            synchronized (this) {
+                notifyAll();
+            }
+        } catch (Exception e) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Unable to stop adapter");
+            }
+        }
+    }
+
+    @Override
+    public FeedConnectionId getFeedId() {
+        return feedId;
+    }
+
+    @Override
+    public String toString() {
+        return feedId + "[" + partition + "]";
+    }
+
+    public IFeedAdapter getFeedAdapter() {
+        return feedAdapter;
+    }
+
+    public void setFeedAdapter(IFeedAdapter feedAdapter) {
+        this.feedAdapter = feedAdapter;
+    }
+
+    public static class AdapterExecutor implements Runnable {
+
+        private FeedFrameWriter writer;
+
+        private IFeedAdapter adapter;
+
+        private AdapterRuntimeManager runtimeManager;
+
+        public AdapterExecutor(int partition, FeedFrameWriter writer, IFeedAdapter adapter,
+                AdapterRuntimeManager adapterRuntimeMgr) {
+            this.writer = writer;
+            this.adapter = adapter;
+            this.runtimeManager = adapterRuntimeMgr;
+        }
+
+        @Override
+        public void run() {
+            try {
+                int partition = runtimeManager.getPartition();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Starting ingestion for partition:" + partition);
+                }
+                adapter.start(partition, writer);
+                runtimeManager.setState(State.FINISHED_INGESTION);
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.SEVERE)) {
+                    LOGGER.severe("Exception during feed ingestion " + e.getMessage());
+                }
+                e.printStackTrace();
+            } finally {
+                synchronized (runtimeManager) {
+                    runtimeManager.notifyAll();
+                }
+            }
+        }
+
+        public FeedFrameWriter getWriter() {
+            return writer;
+        }
+
+        public void setWriter(IFrameWriter writer) {
+            if (this.writer != null) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Switching writer to:" + writer + " from " + this.writer);
+                }
+                this.writer.setWriter(writer);
+            }
+        }
+
+    }
+
+    public synchronized State getState() {
+        return state;
+    }
+
+    @SuppressWarnings("incomplete-switch")
+    public synchronized void setState(State state) throws HyracksDataException {
+        if (this.state.equals(state)) {
+            return;
+        }
+        switch (state) {
+            case INACTIVE_INGESTION:
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Set " + Mode.STORE + " mode");
+                }
+                adapterExecutor.getWriter().setMode(Mode.STORE);
+                break;
+            case ACTIVE_INGESTION:
+                adapterExecutor.getWriter().setMode(Mode.FORWARD);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Set " + Mode.FORWARD + " mode");
+                }
+                break;
+        }
+        this.state = state;
+    }
+
+    public AdapterExecutor getAdapterExecutor() {
+        return adapterExecutor;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    public IngestionRuntime getIngestionRuntime() {
+        return ingestionRuntime;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java
new file mode 100644
index 0000000..3db5916
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java
@@ -0,0 +1,107 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+
+public class BuiltinFeedPolicies {
+
+    public static final FeedPolicy BRITTLE = initializeBrittlePolicy();
+
+    public static final FeedPolicy BASIC = initializeBasicPolicy();
+
+    public static final FeedPolicy BASIC_MONITORED = initializeBasicMonitoredPolicy();
+
+    public static final FeedPolicy FAULT_TOLERANT_BASIC_MONITORED = initializeFaultTolerantBasicMonitoredPolicy();
+
+    public static final FeedPolicy ELASTIC = initializeFaultTolerantBasicMonitoredElasticPolicy();
+
+    public static final FeedPolicy[] policies = new FeedPolicy[] { BRITTLE, BASIC, BASIC_MONITORED,
+            FAULT_TOLERANT_BASIC_MONITORED, ELASTIC };
+
+    public static final FeedPolicy DEFAULT_POLICY = FAULT_TOLERANT_BASIC_MONITORED;
+
+    public static final String CONFIG_FEED_POLICY_KEY = "policy";
+
+    public static FeedPolicy getFeedPolicy(String policyName) {
+        for (FeedPolicy policy : policies) {
+            if (policy.getPolicyName().equalsIgnoreCase(policyName)) {
+                return policy;
+            }
+        }
+        return null;
+    }
+
+    private static FeedPolicy initializeFaultTolerantBasicMonitoredElasticPolicy() {
+        Map<String, String> policyParams = new HashMap<String, String>();
+        policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+        policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+        policyParams.put(FeedPolicyAccessor.ELASTIC, "true");
+        String description = "Basic Monitored Fault-Tolerant Elastic";
+        return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BMFE", description, policyParams);
+    }
+
+    private static FeedPolicy initializeFaultTolerantBasicMonitoredPolicy() {
+        Map<String, String> policyParams = new HashMap<String, String>();
+        policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+        policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+        String description = "Basic Monitored Fault-Tolerant";
+        return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BMF", description, policyParams);
+    }
+
+    private static FeedPolicy initializeBasicMonitoredPolicy() {
+        Map<String, String> policyParams = new HashMap<String, String>();
+        policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "false");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+        policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "false");
+        policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+        String description = "Basic Monitored Fault-Tolerant";
+        return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BM", description, policyParams);
+    }
+
+    private static FeedPolicy initializeBasicPolicy() {
+        Map<String, String> policyParams = new HashMap<String, String>();
+        policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "false");
+        policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "false");
+        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+        String description = "Basic";
+        return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "B", description, policyParams);
+    }
+
+    private static FeedPolicy initializeBrittlePolicy() {
+        Map<String, String> policyParams = new HashMap<String, String>();
+        policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "false");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "false");
+        policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "false");
+        policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "false");
+        policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "false");
+        policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "false");
+        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+        String description = "Brittle";
+        return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "Br", description, policyParams);
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
new file mode 100644
index 0000000..2dfe886
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
@@ -0,0 +1,205 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class ConditionalPushTupleParserFactory implements ITupleParserFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(ConditionalPushTupleParserFactory.class.getName());
+
+    @Override
+    public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+        IDataParser dataParser = null;
+        switch (parserType) {
+            case ADM:
+                dataParser = new ADMDataParser();
+                break;
+            case DELIMITED_DATA:
+                dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+                break;
+        }
+        return new ConditionalPushTupleParser(ctx, recordType, dataParser, configuration);
+    }
+
+    private final ARecordType recordType;
+    private final Map<String, String> configuration;
+    private IValueParserFactory[] valueParserFactories;
+    private char delimiter;
+    private final ParserType parserType;
+    private Object lock;
+
+    public enum ParserType {
+        ADM,
+        DELIMITED_DATA
+    }
+
+    public ConditionalPushTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
+            char fieldDelimiter, Map<String, String> configuration) {
+        this.recordType = recordType;
+        this.valueParserFactories = valueParserFactories;
+        this.delimiter = fieldDelimiter;
+        this.configuration = configuration;
+        this.parserType = ParserType.DELIMITED_DATA;
+
+    }
+
+    public ConditionalPushTupleParserFactory(ARecordType recordType, Map<String, String> configuration) {
+        this.recordType = recordType;
+        this.configuration = configuration;
+        this.parserType = ParserType.ADM;
+    }
+
+}
+
+class ConditionalPushTupleParser extends AbstractTupleParser {
+
+    private final IDataParser dataParser;
+    private int batchSize;
+    private long batchInterval;
+    private boolean continueIngestion = true;
+    private int tuplesInFrame = 0;
+    private TimeBasedFlushTask flushTask;
+    private Timer timer = new Timer();
+    private Object lock = new Object();
+    private boolean activeTimer = false;
+
+    public static final String BATCH_SIZE = "batch-size";
+    public static final String BATCH_INTERVAL = "batch-interval";
+
+    public ConditionalPushTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
+            Map<String, String> configuration) throws HyracksDataException {
+        super(ctx, recType);
+        this.dataParser = dataParser;
+        String propValue = (String) configuration.get(BATCH_SIZE);
+        batchSize = propValue != null ? Integer.parseInt(propValue) : Integer.MAX_VALUE;
+        propValue = (String) configuration.get(BATCH_INTERVAL);
+        batchInterval = propValue != null ? Long.parseLong(propValue) : -1;
+        activeTimer = batchInterval > 0;
+    }
+
+    public void stop() {
+        continueIngestion = false;
+    }
+
+    @Override
+    public IDataParser getDataParser() {
+        return dataParser;
+    }
+
+    @Override
+    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+        flushTask = new TimeBasedFlushTask(writer, lock);
+        appender.reset(frame, true);
+        IDataParser parser = getDataParser();
+        try {
+            parser.initialize(in, recType, true);
+            if (activeTimer) {
+                timer.schedule(flushTask, 0, batchInterval);
+            }
+            while (continueIngestion) {
+                tb.reset();
+                if (!parser.parse(tb.getDataOutput())) {
+                    break;
+                }
+                tb.addFieldEndOffset();
+                addTuple(writer);
+            }
+            if (appender.getTupleCount() > 0) {
+                if (activeTimer) {
+                    synchronized (lock) {
+                        FrameUtils.flushFrame(frame, writer);
+                    }
+                } else {
+                    FrameUtils.flushFrame(frame, writer);
+                }
+            }
+        } catch (AsterixException ae) {
+            throw new HyracksDataException(ae);
+        } catch (IOException ioe) {
+            throw new HyracksDataException(ioe);
+        } finally {
+            if (activeTimer) {
+                timer.cancel();
+            }
+        }
+    }
+
+    protected void addTuple(IFrameWriter writer) throws HyracksDataException {
+        if (activeTimer) {
+            synchronized (lock) {
+                addTupleToFrame(writer);
+            }
+        } else {
+            addTupleToFrame(writer);
+        }
+    }
+
+    protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+        if (tuplesInFrame == batchSize || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+            FrameUtils.flushFrame(frame, writer);
+            appender.reset(frame, true);
+            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                throw new IllegalStateException();
+            }
+            if (tuplesInFrame == batchSize) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Batch size exceeded! flushing frame " + "(" + tuplesInFrame + ")");
+                }
+            }
+            tuplesInFrame = 0;
+        }
+        tuplesInFrame++;
+    }
+
+    private class TimeBasedFlushTask extends TimerTask {
+
+        private IFrameWriter writer;
+        private final Object lock;
+
+        public TimeBasedFlushTask(IFrameWriter writer, Object lock) {
+            this.writer = writer;
+            this.lock = lock;
+        }
+
+        @Override
+        public void run() {
+            try {
+                if (tuplesInFrame > 0) {
+                    synchronized (lock) {
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("TTL expired flushing frame (" + tuplesInFrame + ")");
+                        }
+                        FrameUtils.flushFrame(frame, writer);
+                        appender.reset(frame, true);
+                        tuplesInFrame = 0;
+                    }
+                }
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+        }
+
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java
new file mode 100644
index 0000000..fffee64
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator invovled in the feed pipeline.
+ */
+public class EndFeedMessage extends FeedMessage {
+
+    private static final long serialVersionUID = 1L;
+
+    private final FeedConnectionId feedId;
+
+    public EndFeedMessage(FeedConnectionId feedId) {
+        super(MessageType.END, feedId);
+        this.feedId = feedId;
+    }
+
+    public FeedConnectionId getFeedId() {
+        return feedId;
+    }
+
+    @Override
+    public String toString() {
+        return "" + feedId;
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
similarity index 72%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
index 4290132..c5937f6 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
@@ -12,13 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.data.operator;
+package edu.uci.ics.asterix.metadata.feeds;
 
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -36,17 +31,13 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final Map<String, Object> adapterConfiguration;
-    private final IAType atype;
-    private IGenericDatasetAdapterFactory datasourceAdapterFactory;
+    private IAdapterFactory adapterFactory;
 
-    public ExternalDataScanOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
-            RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory) {
+    public ExternalDataScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
+            IAdapterFactory dataSourceAdapterFactory) {
         super(spec, 0, 1);
         recordDescriptors[0] = rDesc;
-        this.adapterConfiguration = arguments;
-        this.atype = atype;
-        this.datasourceAdapterFactory = dataSourceAdapterFactory;
+        this.adapterFactory = dataSourceAdapterFactory;
     }
 
     @Override
@@ -60,9 +51,7 @@
                 writer.open();
                 IDatasourceAdapter adapter = null;
                 try {
-                    adapter = ((IGenericDatasetAdapterFactory) datasourceAdapterFactory).createAdapter(
-                            adapterConfiguration, atype);
-                    adapter.initialize(ctx);
+                    adapter = ((IAdapterFactory) adapterFactory).createAdapter(ctx);
                     adapter.start(partition, writer);
                 } catch (Exception e) {
                     throw new HyracksDataException("exception during reading from external data source", e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java
new file mode 100644
index 0000000..db1a62a
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class FeedActivityIdFactory {
+	private static AtomicInteger id = new AtomicInteger();
+	private static boolean isInitialized = false;
+
+	public static boolean isInitialized() {
+		return isInitialized;
+	}
+
+	public static void initialize(int initialId) {
+		id.set(initialId);
+		isInitialized = true;
+	}
+
+	public static int generateFeedActivityId() {
+		return id.incrementAndGet();
+	}
+
+	public static int getMostRecentFeedActivityId() {
+		return id.get();
+	}
+}
\ No newline at end of file
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/FeedFrameWriter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
new file mode 100644
index 0000000..53038d2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
@@ -0,0 +1,343 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager.FeedReportMessageType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * A wrapper around the standard frame writer provided to an operator node pushable.
+ * The wrapper monitors the flow of data from this operator to a downstream operator
+ * over a connector. It collects statistics if required by the feed ingestion policy
+ * and reports them to the Super Feed Manager chosen for the feed. In addition any
+ * congestion experienced by the operator is also reported.
+ */
+public class FeedFrameWriter implements IFrameWriter {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedFrameWriter.class.getName());
+
+    /** The threshold for the time required in pushing a frame to the network. **/
+    public static final long FLUSH_THRESHOLD_TIME = 5000; // 3 seconds
+
+    /** Actual frame writer provided to an operator. **/
+    private IFrameWriter writer;
+
+    /** The node pushable associated with the operator **/
+    private IOperatorNodePushable nodePushable;
+
+    /** set to true if health need to be monitored **/
+    private final boolean reportHealth;
+
+    /** A buffer for keeping frames that are waiting to be processed **/
+    private List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+
+    /**
+     * Mode associated with the frame writer
+     * Possible values: FORWARD, STORE
+     * 
+     * @see Mode
+     */
+    private Mode mode;
+
+    /**
+     * Detects if the operator is unable to push a frame downstream
+     * within a threshold period of time. In addition, measure the
+     * throughput as observed on the output channel of the associated operator.
+     */
+    private HealthMonitor healthMonitor;
+
+    /**
+     * Manager scheduling of tasks
+     */
+    private Timer timer;
+
+    /**
+     * Provides access to the tuples in a frame. Used in collecting statistics.
+     */
+    private FrameTupleAccessor fta;
+
+    public enum Mode {
+        /**
+         * **
+         * Normal mode of operation for an operator when
+         * frames are pushed to the downstream operator.
+         */
+        FORWARD,
+
+        /**
+         * Failure mode of operation for an operator when
+         * input frames are not pushed to the downstream operator but
+         * are buffered for future retrieval.
+         */
+        STORE
+    }
+
+    public FeedFrameWriter(IFrameWriter writer, IOperatorNodePushable nodePushable, FeedConnectionId feedId,
+            FeedPolicyEnforcer policyEnforcer, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
+            FrameTupleAccessor fta) {
+        this.writer = writer;
+        this.mode = Mode.FORWARD;
+        this.nodePushable = nodePushable;
+        this.reportHealth = policyEnforcer.getFeedPolicyAccessor().collectStatistics();
+        if (reportHealth) {
+            timer = new Timer();
+            healthMonitor = new HealthMonitor(FLUSH_THRESHOLD_TIME, feedId, nodeId, feedRuntimeType, partition,
+                    FLUSH_THRESHOLD_TIME, timer);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Statistics collection enabled for the feed " + feedId + " " + feedRuntimeType + " ["
+                        + partition + "]");
+            }
+            timer.scheduleAtFixedRate(healthMonitor, 0, FLUSH_THRESHOLD_TIME);
+        } else {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Statistics collection *not* enabled for the feed " + feedId + " " + feedRuntimeType + " ["
+                        + partition + "]");
+            }
+        }
+        this.fta = fta;
+    }
+
+    public Mode getMode() {
+        return mode;
+    }
+
+    public void setMode(Mode newMode) throws HyracksDataException {
+        if (this.mode.equals(newMode)) {
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Switching to :" + newMode + " from " + this.mode);
+        }
+        this.mode = newMode;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        switch (mode) {
+            case FORWARD:
+                try {
+                    if (reportHealth) {
+                        fta.reset(buffer);
+                        healthMonitor.notifyStart();
+                        writer.nextFrame(buffer);
+                        healthMonitor.notifyFinish(fta.getTupleCount());
+                    } else {
+                        writer.nextFrame(buffer);
+                    }
+                } catch (Exception e) {
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Unable to write frame " + " on behalf of " + nodePushable.getDisplayName());
+                    }
+                }
+                if (frames.size() > 0) {
+                    for (ByteBuffer buf : frames) {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Flusing OLD frame: " + buf + " on behalf of "
+                                    + nodePushable.getDisplayName());
+                        }
+                        writer.nextFrame(buf);
+                    }
+                }
+                frames.clear();
+                break;
+            case STORE:
+                ByteBuffer storageBuffer = ByteBuffer.allocate(buffer.capacity());
+                storageBuffer.put(buffer);
+                frames.add(storageBuffer);
+                storageBuffer.flip();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Stored frame for " + nodePushable);
+                }
+                break;
+        }
+    }
+
+    private static class HealthMonitor extends TimerTask {
+
+        private long startTime = -1;
+        private FramePushState state;
+        private long flushThresholdTime;
+        private static final String EOL = "\n";
+        private FeedConnectionId feedId;
+        private String nodeId;
+        private FeedRuntimeType feedRuntimeType;
+        private int partition;
+        private AtomicLong numTuplesInInterval = new AtomicLong(0);
+        private long period;
+        private boolean collectThroughput;
+        private FeedMessageService mesgService;
+
+        public HealthMonitor(long flushThresholdTime, FeedConnectionId feedId, String nodeId,
+                FeedRuntimeType feedRuntimeType, int partition, long period, Timer timer) {
+            this.flushThresholdTime = flushThresholdTime;
+            this.state = FramePushState.INTIALIZED;
+            this.feedId = feedId;
+            this.nodeId = nodeId;
+            this.feedRuntimeType = feedRuntimeType;
+            this.partition = partition;
+            this.period = period;
+            this.collectThroughput = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
+        }
+
+        public void notifyStart() {
+            startTime = System.currentTimeMillis();
+            state = FramePushState.WAITING_FOR_FLUSH_COMPLETION;
+        }
+
+        /**
+         * Reset method is invoked when a live instance of operator needs to take
+         * over from the zombie instance from the previously failed execution
+         */
+        public void reset() {
+            mesgService = null;
+            collectThroughput = true;
+        }
+
+        public void notifyFinish(int numTuples) {
+            state = FramePushState.WAITNG_FOR_NEXT_FRAME;
+            numTuplesInInterval.set(numTuplesInInterval.get() + numTuples);
+        }
+
+        @Override
+        public void run() {
+            if (state.equals(FramePushState.WAITING_FOR_FLUSH_COMPLETION)) {
+                long currentTime = System.currentTimeMillis();
+                if (currentTime - startTime > flushThresholdTime) {
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Congestion reported by " + feedRuntimeType + " [" + partition + "]");
+                    }
+                    sendReportToSFM(currentTime - startTime, FeedReportMessageType.CONGESTION,
+                            System.currentTimeMillis());
+                }
+            }
+            if (collectThroughput) {
+                int instantTput = (int) Math.ceil((((double) numTuplesInInterval.get() * 1000) / period));
+                System.out.println("MEASURED TPUT:" + numTuplesInInterval.get());
+                sendReportToSFM(instantTput, FeedReportMessageType.THROUGHPUT, System.currentTimeMillis());
+            }
+            numTuplesInInterval.set(0);
+        }
+
+        private void sendReportToSFM(long value, SuperFeedManager.FeedReportMessageType mesgType, long timestamp) {
+            if (mesgService == null) {
+                waitTillMessageServiceIsUp();
+            }
+            String feedRep = feedId.getDataverse() + ":" + feedId.getFeedName() + ":" + feedId.getDatasetName();
+            String message = mesgType.name().toLowerCase() + "|" + feedRep + "|" + feedRuntimeType + "|" + partition
+                    + "|" + value + "|" + nodeId + "|" + timestamp + "|" + EOL;
+            try {
+                mesgService.sendMessage(message);
+            } catch (IOException ioe) {
+                ioe.printStackTrace();
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Unable to send feed report to SFM for feed " + feedId + " " + feedRuntimeType + "["
+                            + partition + "]");
+                }
+            }
+        }
+
+        private void waitTillMessageServiceIsUp() {
+            while (mesgService == null) {
+                mesgService = FeedManager.INSTANCE.getFeedMessageService(feedId);
+                if (mesgService == null) {
+                    try {
+                        /**
+                         * wait for the message service to be available
+                         */
+                        Thread.sleep(2000);
+                    } catch (InterruptedException e) {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Encountered an interrupted exception " + " Exception " + e);
+                        }
+                    }
+                }
+            }
+        }
+
+        public void deactivate() {
+            this.cancel();
+            collectThroughput = false;
+        }
+
+        private enum FramePushState {
+            /**
+             * Frame writer has been initialized
+             */
+            INTIALIZED,
+
+            /**
+             * Frame writer is waiting for a pending flush to finish.
+             */
+            WAITING_FOR_FLUSH_COMPLETION,
+
+            /**
+             * Frame writer is waiting to be given the next frame.
+             */
+            WAITNG_FOR_NEXT_FRAME
+        }
+
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+        if (healthMonitor != null && !healthMonitor.feedRuntimeType.equals(FeedRuntimeType.INGESTION)) {
+            healthMonitor.deactivate();
+        }
+        healthMonitor.reset();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (healthMonitor != null) {
+            healthMonitor.deactivate();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Closing frame statistics collection activity" + healthMonitor);
+            }
+        }
+        writer.close();
+    }
+
+    public IFrameWriter getWriter() {
+        return writer;
+    }
+
+    public void setWriter(IFrameWriter writer) {
+        this.writer = writer;
+    }
+
+    @Override
+    public String toString() {
+        return "MaterializingFrameWriter using " + writer;
+    }
+
+    public List<ByteBuffer> getStoredFrames() {
+        return frames;
+    }
+
+    public void clear() {
+        frames.clear();
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+    }
+
+    public void reset() {
+        healthMonitor.reset();
+    }
+
+}
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
new file mode 100644
index 0000000..5c8c422
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2012 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.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+
+/**
+ * FeedIntakeOperatorDescriptor is responsible for ingesting data from an external source. This
+ * operator uses a user specified for a built-in adaptor for retrieving data from the external 
+ * data source. 
+ */
+public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(FeedIntakeOperatorDescriptor.class.getName());
+
+    /** The type associated with the ADM data output from the feed adaptor*/
+    private final IAType atype;
+    
+    /** unique identifier for a feed instance.*/
+    private final FeedConnectionId feedId;
+    
+    /** Map representation of policy parameters */
+    private final Map<String, String> feedPolicy;
+    
+    /** The adaptor factory that is used to create an instance of the feed adaptor**/
+    private IAdapterFactory adapterFactory;
+
+    public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, IAdapterFactory adapterFactory,
+            ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicy) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = rDesc;
+        this.adapterFactory = adapterFactory;
+        this.atype = atype;
+        this.feedId = feedId;
+        this.feedPolicy = feedPolicy;
+    }
+
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+            throws HyracksDataException {
+        IFeedAdapter adapter;
+        FeedRuntimeId feedRuntimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
+        IngestionRuntime ingestionRuntime = (IngestionRuntime) FeedManager.INSTANCE.getFeedRuntime(feedRuntimeId);
+        try {
+            if (ingestionRuntime == null) {
+                adapter = (IFeedAdapter) adapterFactory.createAdapter(ctx);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Beginning new feed:" + feedId);
+                }
+            } else {
+                adapter = ((IngestionRuntime) ingestionRuntime).getAdapterRuntimeManager().getFeedAdapter();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Resuming old feed:" + feedId);
+                }
+            }
+        } catch (Exception e) {
+            if(LOGGER.isLoggable(Level.SEVERE)){
+                LOGGER.severe("Initialization of the feed adaptor failed");
+            }
+            throw new HyracksDataException("initialization of adapter failed", e);
+        }
+        return new FeedIntakeOperatorNodePushable(ctx, feedId, adapter, feedPolicy, partition, ingestionRuntime);
+    }
+
+    public FeedConnectionId getFeedId() {
+        return feedId;
+    }
+
+    public Map<String, String> getFeedPolicy() {
+        return feedPolicy;
+    }
+
+    public IAdapterFactory getAdapterFactory() {
+        return adapterFactory;
+    }
+
+    public IAType getAtype() {
+        return atype;
+    }
+
+    public RecordDescriptor getRecordDescriptor() {
+        return recordDescriptors[0];
+    }
+}
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
new file mode 100644
index 0000000..9a70ca3
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -0,0 +1,164 @@
+/*
+ * 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.IOException;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager.State;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+/**
+ * The runtime for @see{FeedIntakeOperationDescriptor}
+ */
+public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+
+    private static Logger LOGGER = Logger.getLogger(FeedIntakeOperatorNodePushable.class.getName());
+
+    private IFeedAdapter adapter;
+    private final int partition;
+    private final FeedConnectionId feedId;
+    private final LinkedBlockingQueue<IFeedMessage> inbox;
+    private final Map<String, String> feedPolicy;
+    private final FeedPolicyEnforcer policyEnforcer;
+    private FeedRuntime ingestionRuntime;
+    private final String nodeId;
+    private FrameTupleAccessor fta;
+    private FeedFrameWriter feedFrameWriter;
+
+    public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedAdapter adapter,
+            Map<String, String> feedPolicy, int partition, IngestionRuntime ingestionRuntime) {
+        this.adapter = adapter;
+        this.partition = partition;
+        this.feedId = feedId;
+        this.ingestionRuntime = ingestionRuntime;
+        inbox = new LinkedBlockingQueue<IFeedMessage>();
+        this.feedPolicy = feedPolicy;
+        policyEnforcer = new FeedPolicyEnforcer(feedId, feedPolicy);
+        nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
+        fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+
+        AdapterRuntimeManager adapterRuntimeMgr = null;
+        try {
+            if (ingestionRuntime == null) {
+                feedFrameWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
+                        FeedRuntimeType.INGESTION, partition, fta);
+                adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox);
+
+                if (adapter instanceof AbstractFeedDatasourceAdapter) {
+                    ((AbstractFeedDatasourceAdapter) adapter).setFeedPolicyEnforcer(policyEnforcer);
+                }
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Beginning new feed:" + feedId);
+                }
+                feedFrameWriter.open();
+                adapterRuntimeMgr.start();
+            } else {
+                adapterRuntimeMgr = ((IngestionRuntime) ingestionRuntime).getAdapterRuntimeManager();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Resuming old feed:" + feedId);
+                }
+                adapter = adapterRuntimeMgr.getFeedAdapter();
+                writer.open();
+                adapterRuntimeMgr.getAdapterExecutor().setWriter(writer);
+                adapterRuntimeMgr.getAdapterExecutor().getWriter().reset();
+                adapterRuntimeMgr.setState(State.ACTIVE_INGESTION);
+                feedFrameWriter = adapterRuntimeMgr.getAdapterExecutor().getWriter();
+            }
+
+            ingestionRuntime = adapterRuntimeMgr.getIngestionRuntime();
+            synchronized (adapterRuntimeMgr) {
+                while (!adapterRuntimeMgr.getState().equals(State.FINISHED_INGESTION)) {
+                    adapterRuntimeMgr.wait();
+                }
+            }
+            FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+            feedFrameWriter.close();
+        } catch (InterruptedException ie) {
+            if (policyEnforcer.getFeedPolicyAccessor().continueOnHardwareFailure()) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Continuing on failure as per feed policy, switching to INACTIVE INGESTION temporarily");
+                }
+                adapterRuntimeMgr.setState(State.INACTIVE_INGESTION);
+                FeedRuntimeManager runtimeMgr = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+                try {
+                    runtimeMgr.close(false);
+                } catch (IOException ioe) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Unable to close Feed Runtime Manager " + ioe.getMessage());
+                    }
+                }
+                feedFrameWriter.fail();
+            } else {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Interrupted Exception, something went wrong");
+                }
+               
+                FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+                feedFrameWriter.close();
+                throw new HyracksDataException(ie);
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public Map<String, String> getFeedPolicy() {
+        return feedPolicy;
+    }
+}
+
+class FeedInboxMonitor extends Thread {
+
+    private LinkedBlockingQueue<IFeedMessage> inbox;
+    private final AdapterRuntimeManager runtimeMgr;
+
+    public FeedInboxMonitor(AdapterRuntimeManager runtimeMgr, LinkedBlockingQueue<IFeedMessage> inbox, int partition) {
+        this.inbox = inbox;
+        this.runtimeMgr = runtimeMgr;
+    }
+
+    @Override
+    public void run() {
+        while (true) {
+            try {
+                IFeedMessage feedMessage = inbox.take();
+                switch (feedMessage.getMessageType()) {
+                    case END:
+                        runtimeMgr.stop();
+                        break;
+                }
+            } catch (InterruptedException ie) {
+                break;
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..d910ca8
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
@@ -0,0 +1,129 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+public class FeedManager implements IFeedManager {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedManager.class.getName());
+
+    public static final long SOCKET_CONNECT_TIMEOUT = 5000;
+
+    public static FeedManager INSTANCE = new FeedManager();
+
+    private FeedManager() {
+
+    }
+
+    private Map<FeedConnectionId, FeedRuntimeManager> feedRuntimeManagers = new HashMap<FeedConnectionId, FeedRuntimeManager>();
+
+    public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
+        return feedRuntimeManagers.get(feedId);
+    }
+
+    public ExecutorService getFeedExecutorService(FeedConnectionId feedId) {
+        FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+        return mgr == null ? null : mgr.getExecutorService();
+    }
+
+    public FeedMessageService getFeedMessageService(FeedConnectionId feedId) {
+        FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+        return mgr == null ? null : mgr.getMessageService();
+    }
+
+    @Override
+    public void deregisterFeed(FeedConnectionId feedId) {
+        try {
+            FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+            if (mgr == null) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("unknown feed id: " + feedId);
+                }
+            } else {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Closing feed runtime manager: " + mgr);
+                }
+                mgr.close(true);
+            }
+        } catch (Exception e) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Exception in closing feed runtime" + e.getMessage());
+            }
+            e.printStackTrace();
+        }
+
+        feedRuntimeManagers.remove(feedId);
+    }
+
+    @Override
+    public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception {
+        FeedConnectionId feedId = feedRuntime.getFeedRuntimeId().getFeedId();
+        FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+        if (runtimeMgr == null) {
+            synchronized (feedRuntimeManagers) {
+                if (runtimeMgr == null) {
+                    runtimeMgr = new FeedRuntimeManager(feedId);
+                    feedRuntimeManagers.put(feedId, runtimeMgr);
+                }
+            }
+        }
+
+        runtimeMgr.registerFeedRuntime(feedRuntime.getFeedRuntimeId(), feedRuntime);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Registered runtime " + feedRuntime + " for feed " + feedId);
+        }
+    }
+
+    @Override
+    public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId) {
+        FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedRuntimeId.getFeedId());
+        if (runtimeMgr != null) {
+            runtimeMgr.deregisterFeedRuntime(feedRuntimeId);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Deregistered Feed Runtime " + feedRuntimeId);
+            }
+        }
+    }
+
+    @Override
+    public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId) {
+        FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedRuntimeId.getFeedId());
+        return runtimeMgr != null ? runtimeMgr.getFeedRuntime(feedRuntimeId) : null;
+    }
+
+    @Override
+    public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception {
+        FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+        if (runtimeMgr != null) {
+            runtimeMgr.setSuperFeedManager(sfm);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Registered Super Feed Manager " + sfm);
+            }
+        }
+    }
+
+    @Override
+    public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId) {
+        FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+        return runtimeMgr != null ? runtimeMgr.getSuperFeedManager() : null;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java
new file mode 100644
index 0000000..aedd124
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+/**
+ * A feed control message containing the altered values for
+ * adapter configuration parameters. This message is dispatched
+ * to all runtime instances of the feed's adapter.
+ */
+public class FeedManagerElectMessage extends FeedMessage {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String host;
+    private final String nodeId;
+    private final int port;
+
+    public FeedManagerElectMessage(String host, String nodeId, int port, FeedConnectionId feedId) {
+        super(MessageType.SUPER_FEED_MANAGER_ELECT, feedId);
+        this.host = host;
+        this.port = port;
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    public MessageType getMessageType() {
+        return MessageType.SUPER_FEED_MANAGER_ELECT;
+    }
+
+    @Override
+    public String toString() {
+        return host + "_" + nodeId + "[" + port + "]";
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
similarity index 71%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
index 96262fe..36fcf4b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
@@ -12,28 +12,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.feeds;
+
+import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
 
 /**
- * A control message that can be sent to the runtime instance of a 
- * feed's adapter. 
+ * A control message that can be sent to the runtime instance of a
+ * feed's adapter.
  */
 public class FeedMessage implements IFeedMessage {
 
     private static final long serialVersionUID = 1L;
 
-    protected MessageType messageType;
+    protected final MessageType messageType;
+    protected final FeedConnectionId feedId;
 
-    public FeedMessage(MessageType messageType) {
+    public FeedMessage(MessageType messageType, FeedConnectionId feedId) {
         this.messageType = messageType;
+        this.feedId = feedId;
     }
 
     public MessageType getMessageType() {
         return messageType;
     }
 
-    public void setMessageType(MessageType messageType) {
-        this.messageType = messageType;
+    public FeedConnectionId getFeedId() {
+        return feedId;
     }
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
similarity index 74%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
index 300e3af..fae2fe4 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
@@ -12,12 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.data.operator;
+package edu.uci.ics.asterix.metadata.feeds;
 
-import java.util.List;
-
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -32,21 +28,22 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final FeedId feedId;
-    private final List<IFeedMessage> feedMessages;
-    private final boolean sendToAll = true;
+    private final FeedConnectionId feedId;
+    private final IFeedMessage feedMessage;
 
-    public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String dataset,
-            List<IFeedMessage> feedMessages) {
+    public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String feedName, String dataset,
+            IFeedMessage feedMessage) {
         super(spec, 0, 1);
-        this.feedId = new FeedId(dataverse, dataset);
-        this.feedMessages = feedMessages;
+        this.feedId = new FeedConnectionId(dataverse, feedName, dataset);
+        this.feedMessage = feedMessage;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new FeedMessageOperatorNodePushable(ctx, feedId, feedMessages, sendToAll, partition, nPartitions);
+        return new FeedMessageOperatorNodePushable(ctx, feedId, feedMessage, partition, nPartitions);
     }
 
+   
+
 }
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
new file mode 100644
index 0000000..a8541f5
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
@@ -0,0 +1,101 @@
+/*
+ * 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.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+/**
+ * Runtime for the @see{FeedMessageOperatorDescriptor}
+ */
+public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedMessageOperatorNodePushable.class.getName());
+
+    private final FeedConnectionId feedId;
+    private final IFeedMessage feedMessage;
+    private final int partition;
+    private final IHyracksTaskContext ctx;
+
+    public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedMessage feedMessage,
+            int partition, int nPartitions) {
+        this.feedId = feedId;
+        this.feedMessage = feedMessage;
+        this.partition = partition;
+        this.ctx = ctx;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        try {
+            writer.open();
+            FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
+            FeedRuntime feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+            boolean ingestionLocation = feedRuntime != null;
+
+            switch (feedMessage.getMessageType()) {
+                case END:
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Ending feed:" + feedId);
+                    }
+
+                    if (ingestionLocation) {
+                        AdapterRuntimeManager adapterRuntimeMgr = ((IngestionRuntime) feedRuntime)
+                                .getAdapterRuntimeManager();
+                        adapterRuntimeMgr.stop();
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Terminating ingestion for :" + feedId);
+                        }
+                    }
+                    break;
+
+                case SUPER_FEED_MANAGER_ELECT:
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Registering SUPER Feed MGR for :" + feedId);
+                    }
+                    FeedManagerElectMessage mesg = ((FeedManagerElectMessage) feedMessage);
+                    SuperFeedManager sfm = new SuperFeedManager(mesg.getFeedId(), mesg.getHost(), mesg.getNodeId(),
+                            mesg.getPort());
+                    synchronized (FeedManager.INSTANCE) {
+                        INCApplicationContext ncCtx = ctx.getJobletContext().getApplicationContext();
+                        String nodeId = ncCtx.getNodeId();
+                        if (sfm.getNodeId().equals(nodeId)) {
+                            sfm.setLocal(true);
+                        } else {
+                            Thread.sleep(5000);
+                        }
+                        FeedManager.INSTANCE.registerSuperFeedManager(feedId, sfm);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("Registered super feed mgr " + sfm + " for feed " + feedId);
+                        }
+                    }
+                    break;
+            }
+
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
new file mode 100644
index 0000000..40b2f26
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.nio.CharBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Sends feed report messages on behalf of an operator instance
+ * to the SuperFeedMaanger associated with the feed.
+ */
+public class FeedMessageService {
+
+    private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
+    private static final char EOL = (char) "\n".getBytes()[0];
+
+    private final FeedConnectionId feedId;
+    private final LinkedBlockingQueue<String> inbox;
+    private final FeedMessageHandler mesgHandler;
+
+    public FeedMessageService(FeedConnectionId feedId) {
+        this.feedId = feedId;
+        this.inbox = new LinkedBlockingQueue<String>();
+        mesgHandler = new FeedMessageHandler(inbox, feedId);
+    }
+
+    public void start() throws UnknownHostException, IOException, Exception {
+        FeedManager.INSTANCE.getFeedExecutorService(feedId).execute(mesgHandler);
+    }
+
+    public void stop() throws IOException {
+        mesgHandler.stop();
+    }
+
+    public void sendMessage(String message) throws IOException {
+        inbox.add(message);
+    }
+
+    private static class FeedMessageHandler implements Runnable {
+
+        private final LinkedBlockingQueue<String> inbox;
+        private final FeedConnectionId feedId;
+        private Socket sfmSocket;
+        private boolean process = true;
+
+        public FeedMessageHandler(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId) {
+            this.inbox = inbox;
+            this.feedId = feedId;
+        }
+
+        public void run() {
+            try {
+                sfmSocket = obtainSFMSocket();
+                if (sfmSocket != null) {
+                    while (process) {
+                        String message = inbox.take();
+                        sfmSocket.getOutputStream().write(message.getBytes());
+                    }
+                } else {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Unable to start feed message service for " + feedId);
+                    }
+                }
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("ENDED FEED MESSAGE SERVICE for " + feedId);
+                }
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Exception in handling incoming feed messages" + e.getMessage());
+                }
+            } finally {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Stopping feed message handler");
+                }
+                if (sfmSocket != null) {
+                    try {
+                        sfmSocket.close();
+                    } catch (Exception e) {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Exception in closing socket " + e.getMessage());
+                        }
+                    }
+                }
+            }
+
+        }
+
+        public void stop() {
+            process = false;
+        }
+
+        private Socket obtainSFMSocket() throws UnknownHostException, IOException, Exception {
+            Socket sfmDirServiceSocket = null;
+            SuperFeedManager sfm = FeedManager.INSTANCE.getSuperFeedManager(feedId);
+            try {
+                FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+                sfmDirServiceSocket = runtimeManager.createClientSocket(sfm.getHost(), sfm.getPort(),
+                        FeedManager.SOCKET_CONNECT_TIMEOUT);
+                if (sfmDirServiceSocket == null) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Unable to connect to " + sfm.getHost() + "[" + sfm.getPort() + "]");
+                    }
+                } else {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info(" Connected to Super Feed Manager service " + sfm.getHost() + " " + sfm.getPort());
+                    }
+                    while (!sfmDirServiceSocket.isConnected()) {
+                        Thread.sleep(2000);
+                    }
+                    InputStream in = sfmDirServiceSocket.getInputStream();
+                    CharBuffer buffer = CharBuffer.allocate(50);
+                    char ch = 0;
+                    while (ch != EOL) {
+                        buffer.put(ch);
+                        ch = (char) in.read();
+                    }
+                    buffer.flip();
+                    String s = new String(buffer.array());
+                    int port = Integer.parseInt(s.trim());
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Response from Super Feed Manager service " + port + " will connect at "
+                                + sfm.getHost() + " " + port);
+                    }
+                    sfmSocket = runtimeManager.createClientSocket(sfm.getHost(), port,
+                            FeedManager.SOCKET_CONNECT_TIMEOUT);
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+                throw e;
+            } finally {
+                if (sfmDirServiceSocket != null) {
+                    sfmDirServiceSocket.close();
+                }
+            }
+            return sfmSocket;
+        }
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
new file mode 100644
index 0000000..bcadc18
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
@@ -0,0 +1,272 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeState;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+
+/**
+ * FeedMetaOperatorDescriptor is a wrapper operator that provides a sanboox like
+ * environment for an hyracks operator that is part of a feed ingestion pipeline.
+ * The MetaFeed operator provides an interface iden- tical to that offered by the
+ * underlying wrapped operator, hereafter referred to as the core operator.
+ * As seen by Hyracks, the altered pipeline is identical to the earlier version formed
+ * from core operators. The MetaFeed operator enhances each core operator by providing
+ * functionality for handling runtime exceptions, saving any state for future retrieval,
+ * and measuring/reporting of performance characteristics. We next describe how the added
+ * functionality contributes to providing fault- tolerance.
+ */
+
+public class FeedMetaOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(FeedMetaOperatorDescriptor.class.getName());
+
+    /** The actual (Hyracks) operator that is wrapped around by the Metafeed Adaptor **/
+    private IOperatorDescriptor coreOperator;
+
+    /**
+     * A unique identifier for the feed instance. A feed instance represents the flow of data
+     * from a feed to a dataset.
+     **/
+    private final FeedConnectionId feedConnectionId;
+
+    /**
+     * The policy associated with the feed instance.
+     */
+    private final FeedPolicy feedPolicy;
+
+    /**
+     * type for the feed runtime associated with the operator.
+     * Possible values: INGESTION, COMPUTE, STORAGE, COMMIT
+     */
+    private final FeedRuntimeType runtimeType;
+
+    public FeedMetaOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId,
+            IOperatorDescriptor coreOperatorDescriptor, FeedPolicy feedPolicy, FeedRuntimeType runtimeType) {
+        super(spec, coreOperatorDescriptor.getInputArity(), coreOperatorDescriptor.getOutputArity());
+        this.feedConnectionId = feedConnectionId;
+        this.feedPolicy = feedPolicy;
+        if (coreOperatorDescriptor.getOutputRecordDescriptors().length == 1) {
+            recordDescriptors[0] = coreOperatorDescriptor.getOutputRecordDescriptors()[0];
+        }
+        this.coreOperator = coreOperatorDescriptor;
+        this.runtimeType = runtimeType;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        return new FeedMetaNodePushable(ctx, recordDescProvider, partition, nPartitions, coreOperator,
+                feedConnectionId, feedPolicy, runtimeType);
+    }
+
+    @Override
+    public String toString() {
+        return "FeedMeta [" + coreOperator + " ]";
+    }
+
+    private static class FeedMetaNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+        /** Runtime node pushable corresponding to the core feed operator **/
+        private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperatorNodePushable;
+
+        /**
+         * A policy enforcer that ensures dyanmic decisions for a feed are taken in accordance
+         * with the associated ingestion policy
+         **/
+        private FeedPolicyEnforcer policyEnforcer;
+
+        /**
+         * The Feed Runtime instance associated with the operator. Feed Runtime captures the state of the operator while
+         * the feed is active.
+         */
+        private FeedRuntime feedRuntime;
+
+        /**
+         * A unique identifier for the feed instance. A feed instance represents the flow of data
+         * from a feed to a dataset.
+         **/
+        private FeedConnectionId feedId;
+
+        /** Denotes the i'th operator instance in a setting where K operator instances are scheduled to run in parallel **/
+        private int partition;
+
+        /** A buffer that is used to hold the current frame that is being processed **/
+        private ByteBuffer currentBuffer;
+
+        /** Type associated with the core feed operator **/
+        private final FeedRuntimeType runtimeType;
+
+        /** True is the feed is recovering from a previous failed execution **/
+        private boolean resumeOldState;
+
+        /** The Node Controller ID for the host NC **/
+
+        private String nodeId;
+
+        /** Allows to iterate over the tuples in a frame **/
+        private FrameTupleAccessor fta;
+
+        public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+                int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
+                FeedPolicy feedPolicy, FeedRuntimeType runtimeType) throws HyracksDataException {
+            this.coreOperatorNodePushable = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+                    .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+            this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicy.getProperties());
+            this.partition = partition;
+            this.runtimeType = runtimeType;
+            this.feedId = feedConnectionId;
+            this.nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
+            fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, feedId, partition);
+            try {
+                feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+                if (feedRuntime == null) {
+                    feedRuntime = new FeedRuntime(feedId, partition, runtimeType);
+                    FeedManager.INSTANCE.registerFeedRuntime(feedRuntime);
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Did not find a saved state from a previous zombie, starting a new instance for "
+                                + runtimeType + " node.");
+                    }
+                    resumeOldState = false;
+                } else {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Retreived state from the zombie instance from previous execution for "
+                                + runtimeType + " node.");
+                    }
+                    resumeOldState = true;
+                }
+                FeedFrameWriter mWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
+                        runtimeType, partition, fta);
+                coreOperatorNodePushable.setOutputFrameWriter(0, mWriter, recordDesc);
+                coreOperatorNodePushable.open();
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.SEVERE)) {
+                    LOGGER.severe("Unable to initialize feed operator " + feedRuntime + " [" + partition + "]");
+                }
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            try {
+                if (resumeOldState) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("State from previous zombie instance "
+                                + feedRuntime.getRuntimeState().getFrame());
+                    }
+                    coreOperatorNodePushable.nextFrame(feedRuntime.getRuntimeState().getFrame());
+                    feedRuntime.setRuntimeState(null);
+                    resumeOldState = false;
+                }
+                coreOperatorNodePushable.nextFrame(buffer);
+            } catch (HyracksDataException e) {
+                if (policyEnforcer.getFeedPolicyAccessor().continueOnApplicationFailure()) {
+                    boolean isExceptionHarmful = handleException(e.getCause());
+                    if (isExceptionHarmful) {
+                        // log the tuple
+                        FeedRuntimeState runtimeState = new FeedRuntimeState(buffer, writer, e);
+                        feedRuntime.setRuntimeState(runtimeState);
+                        String message = e.getMessage();
+                        String tIndexString = message.substring(message.lastIndexOf(':'));
+                        int tupleIndex = 0;
+                        if (tIndexString != null) {
+                            tupleIndex = Integer.parseInt(tIndexString);
+                        }
+                        fta.reset(buffer);
+                        int endOffset = fta.getTupleEndOffset(tupleIndex);
+                        buffer.flip();
+                        buffer.position(endOffset + 1);
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Harmful exception (parked data) tupleIndex " + tupleIndex + e);
+                        }
+                    } else {
+                        // ignore the frame (exception is expected)
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Ignoring exception " + e);
+                        }
+                    }
+                } else {
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Feed policy does not require feed to survive soft failure");
+                    }
+                    throw e;
+                }
+            }
+        }
+
+        private boolean handleException(Throwable exception) {
+            if (exception instanceof TreeIndexException) {
+                if (resumeOldState) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Received duplicate key exception but that is possible post recovery");
+                    }
+                    return false;
+                } else {
+                    if (LOGGER.isLoggable(Level.SEVERE)) {
+                        LOGGER.severe("Received duplicate key exception!");
+                    }
+                    return true;
+                }
+            }
+            return true;
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.info("Core Op:" + coreOperatorNodePushable.getDisplayName() + " fail ");
+            }
+            if (policyEnforcer.getFeedPolicyAccessor().continueOnHardwareFailure()) {
+                if (currentBuffer != null) {
+                    FeedRuntimeState runtimeState = new FeedRuntimeState(currentBuffer, writer, null);
+                    feedRuntime.setRuntimeState(runtimeState);
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Saved feed compute runtime for revivals" + feedRuntime.getFeedRuntimeId());
+                    }
+                } else {
+                    FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning(" No state to save, de-registered feed compute runtime "
+                                + feedRuntime.getFeedRuntimeId());
+                    }
+                }
+            }
+            coreOperatorNodePushable.fail();
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            coreOperatorNodePushable.close();
+            FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+        }
+
+    }
+
+    public IOperatorDescriptor getCoreOperator() {
+        return coreOperator;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java
new file mode 100644
index 0000000..a3c505b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java
@@ -0,0 +1,91 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
+
+public class FeedPolicyAccessor {
+    public static final String FAILURE_LOG_ERROR = "failure.log.error";
+    public static final String APPLICATION_FAILURE_LOG_DATA = "application.failure.log.data";
+    public static final String APPLICATION_FAILURE_CONTINUE = "application.failure.continue";
+    public static final String HARDWARE_FAILURE_CONTINUE = "hardware.failure.continue";
+    public static final String CLUSTER_REBOOT_AUTO_RESTART = "cluster.reboot.auto.restart";
+    public static final String COLLECT_STATISTICS = "collect.statistics";
+    public static final String COLLECT_STATISTICS_PERIOD = "collect.statistics.period";
+    public static final String COLLECT_STATISTICS_PERIOD_UNIT = "collect.statistics.period.unit";
+    public static final String ELASTIC = "elastic";
+
+    public enum TimeUnit {
+        SEC,
+        MIN,
+        HRS,
+        DAYS
+    }
+
+    private Map<String, String> feedPolicy;
+
+    public FeedPolicyAccessor(Map<String, String> feedPolicy) {
+        this.feedPolicy = feedPolicy;
+    }
+
+    public boolean logErrorOnFailure() {
+        return getBooleanPropertyValue(FAILURE_LOG_ERROR);
+    }
+
+    public boolean logDataOnApplicationFailure() {
+        return getBooleanPropertyValue(APPLICATION_FAILURE_LOG_DATA);
+    }
+
+    public boolean continueOnApplicationFailure() {
+        return getBooleanPropertyValue(APPLICATION_FAILURE_CONTINUE);
+    }
+
+    public boolean continueOnHardwareFailure() {
+        return getBooleanPropertyValue(HARDWARE_FAILURE_CONTINUE);
+    }
+
+    public boolean autoRestartOnClusterReboot() {
+        return getBooleanPropertyValue(CLUSTER_REBOOT_AUTO_RESTART);
+    }
+
+    public boolean collectStatistics() {
+        return getBooleanPropertyValue(COLLECT_STATISTICS);
+    }
+
+    public long getStatisicsCollectionPeriodInSecs() {
+        return getIntegerPropertyValue(COLLECT_STATISTICS_PERIOD) * getTimeUnitFactor();
+    }
+
+    public boolean isElastic() {
+        return getBooleanPropertyValue(ELASTIC);
+    }
+
+    private int getTimeUnitFactor() {
+        String v = feedPolicy.get(COLLECT_STATISTICS_PERIOD_UNIT);
+        int factor = 1;
+        switch (TimeUnit.valueOf(v)) {
+            case SEC:
+                factor = 1;
+                break;
+            case MIN:
+                factor = 60;
+                break;
+            case HRS:
+                factor = 3600;
+                break;
+            case DAYS:
+                factor = 216000;
+                break;
+
+        }
+        return factor;
+    }
+
+    private boolean getBooleanPropertyValue(String key) {
+        String v = feedPolicy.get(key);
+        return v == null ? false : Boolean.valueOf(v);
+    }
+
+    private int getIntegerPropertyValue(String key) {
+        String v = feedPolicy.get(key);
+        return Integer.parseInt(v);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..b94e228
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
@@ -0,0 +1,64 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.rmi.RemoteException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+
+public class FeedPolicyEnforcer {
+
+    private final FeedConnectionId feedId;
+    private final FeedPolicyAccessor feedPolicyAccessor;
+    private final FeedActivity feedActivity;
+
+    public FeedPolicyEnforcer(FeedConnectionId feedId, Map<String, String> feedPolicy) {
+        this.feedId = feedId;
+        this.feedPolicyAccessor = new FeedPolicyAccessor(feedPolicy);
+        this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getFeedName(), feedId.getDatasetName(),
+                null, new HashMap<String, String>());
+    }
+
+    public boolean handleSoftwareFailure(Exception e) throws RemoteException, ACIDException {
+        boolean continueIngestion = feedPolicyAccessor.continueOnApplicationFailure();
+        if (feedPolicyAccessor.logErrorOnFailure()) {
+            persistExceptionDetails(e);
+        }
+        return continueIngestion;
+    }
+
+    private synchronized void persistExceptionDetails(Exception e) throws RemoteException, ACIDException {
+        MetadataManager.INSTANCE.acquireWriteLatch();
+        MetadataTransactionContext ctx = null;
+        try {
+            ctx = MetadataManager.INSTANCE.beginTransaction();
+            feedActivity.setActivityType(FeedActivityType.FEED_FAILURE);
+            feedActivity.getFeedActivityDetails().put(FeedActivity.FeedActivityDetails.EXCEPTION_MESSAGE,
+                    e.getMessage());
+            MetadataManager.INSTANCE.registerFeedActivity(ctx, feedId, feedActivity);
+            MetadataManager.INSTANCE.commitTransaction(ctx);
+        } catch (Exception e2) {
+            MetadataManager.INSTANCE.abortTransaction(ctx);
+        } finally {
+            MetadataManager.INSTANCE.releaseWriteLatch();
+        }
+    }
+
+    public void handleHardwareFailure(List<String> nodeId) {
+
+    }
+
+    public FeedPolicyAccessor getFeedPolicyAccessor() {
+        return feedPolicyAccessor;
+    }
+
+    public FeedConnectionId getFeedId() {
+        return feedId;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java
new file mode 100644
index 0000000..fbff931
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java
@@ -0,0 +1,116 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager.FeedReportMessageType;
+
+public class FeedReport implements Comparable {
+
+    private FeedConnectionId feedId;
+    private FeedReportMessageType reportType;
+    private int partition = -1;
+    private FeedRuntimeType runtimeType;
+    private long value = -1;
+    private String[] representation;
+
+    public FeedReport() {
+    }
+
+    public FeedReport(String message) {
+        representation = message.split("\\|");
+    }
+
+    public void reset(String message) {
+        representation = message.split("\\|");
+        reportType = null;
+        feedId = null;
+        runtimeType = null;
+        partition = -1;
+        value = -1;
+    }
+
+    @Override
+    public String toString() {
+        return getFeedId() + " " + getReportType() + " " + getPartition() + " " + getRuntimeType() + " " + getValue();
+    }
+
+    public FeedConnectionId getFeedId() {
+        if (feedId == null) {
+            String feedIdRep = representation[1];
+            String[] feedIdComp = feedIdRep.split(":");
+            feedId = new FeedConnectionId(feedIdComp[0], feedIdComp[1], feedIdComp[2]);
+        }
+        return feedId;
+    }
+
+    public FeedReportMessageType getReportType() {
+        if (reportType == null) {
+            reportType = FeedReportMessageType.valueOf(representation[0].toUpperCase());
+        }
+        return reportType;
+    }
+
+    public int getPartition() {
+        if (partition < 0) {
+            partition = Integer.parseInt(representation[3]);
+        }
+        return partition;
+    }
+
+    public FeedRuntimeType getRuntimeType() {
+        if (runtimeType == null) {
+            runtimeType = FeedRuntimeType.valueOf(representation[2].toUpperCase());
+        }
+        return runtimeType;
+    }
+
+    public long getValue() {
+        if (value < 0) {
+            value = Long.parseLong(representation[4]);
+        }
+        return value;
+    }
+
+    public String[] getRepresentation() {
+        return representation;
+    }
+
+    @Override
+    public int compareTo(Object o) {
+        if (!(o instanceof FeedReport)) {
+            throw new IllegalArgumentException("Incorrect operand type " + o);
+        }
+
+        FeedReport other = (FeedReport) o;
+        if (!other.getReportType().equals(getReportType())) {
+            throw new IllegalArgumentException("Incorrect operand type " + o);
+        }
+
+        int returnValue = 0;
+
+        switch (getReportType()) {
+            case CONGESTION:
+                returnValue = ranking.get(getRuntimeType()) - ranking.get(other.getRuntimeType());
+                break;
+
+            case THROUGHPUT:
+                returnValue = (int) (other.getValue() - getValue());
+                break;
+        }
+
+        return returnValue;
+    }
+
+    private static Map<FeedRuntimeType, Integer> ranking = populateRanking();
+
+    private static Map<FeedRuntimeType, Integer> populateRanking() {
+        Map<FeedRuntimeType, Integer> ranking = new HashMap<FeedRuntimeType, Integer>();
+        ranking.put(FeedRuntimeType.INGESTION, 1);
+        ranking.put(FeedRuntimeType.COMPUTE, 2);
+        ranking.put(FeedRuntimeType.STORAGE, 3);
+        ranking.put(FeedRuntimeType.COMMIT, 4);
+        return ranking;
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
new file mode 100644
index 0000000..af25be2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
@@ -0,0 +1,133 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public class FeedRuntime {
+
+    public enum FeedRuntimeType {
+        INGESTION,
+        COMPUTE,
+        STORAGE,
+        COMMIT
+    }
+
+    protected final FeedRuntimeId feedRuntimeId;
+
+    protected FeedRuntimeState runtimeState;
+
+    public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType) {
+        this.feedRuntimeId = new FeedRuntimeId(feedRuntimeType, feedId, partition);
+    }
+
+    public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType,
+            FeedRuntimeState runtimeState) {
+        this.feedRuntimeId = new FeedRuntimeId(feedRuntimeType, feedId, partition);
+        this.runtimeState = runtimeState;
+    }
+
+    @Override
+    public String toString() {
+        return feedRuntimeId + " " + "runtime state ? " + (runtimeState != null);
+    }
+
+    public static class FeedRuntimeState {
+
+        private ByteBuffer frame;
+        private IFrameWriter frameWriter;
+        private Exception exception;
+
+        public FeedRuntimeState(ByteBuffer frame, IFrameWriter frameWriter, Exception e) {
+            this.frame = frame;
+            this.frameWriter = frameWriter;
+            this.exception = exception;
+        }
+
+        public ByteBuffer getFrame() {
+            return frame;
+        }
+
+        public void setFrame(ByteBuffer frame) {
+            this.frame = frame;
+        }
+
+        public IFrameWriter getFrameWriter() {
+            return frameWriter;
+        }
+
+        public void setFrameWriter(IFrameWriter frameWriter) {
+            this.frameWriter = frameWriter;
+        }
+
+        public Exception getException() {
+            return exception;
+        }
+
+        public void setException(Exception exception) {
+            this.exception = exception;
+        }
+
+    }
+
+    public static class FeedRuntimeId {
+
+        private final FeedRuntimeType feedRuntimeType;
+        private final FeedConnectionId feedId;
+        private final int partition;
+        private final int hashCode;
+
+        public FeedRuntimeId(FeedRuntimeType runtimeType, FeedConnectionId feedId, int partition) {
+            this.feedRuntimeType = runtimeType;
+            this.feedId = feedId;
+            this.partition = partition;
+            this.hashCode = (feedId + "[" + partition + "]" + feedRuntimeType).hashCode();
+        }
+
+        @Override
+        public String toString() {
+            return feedId + "[" + partition + "]" + " " + feedRuntimeType;
+        }
+
+        @Override
+        public int hashCode() {
+            return hashCode;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof FeedRuntimeId) {
+                FeedRuntimeId oid = ((FeedRuntimeId) o);
+                return oid.getFeedId().equals(feedId) && oid.getFeedRuntimeType().equals(feedRuntimeType)
+                        && oid.getPartition() == partition;
+            }
+            return false;
+        }
+
+        public FeedRuntimeType getFeedRuntimeType() {
+            return feedRuntimeType;
+        }
+
+        public FeedConnectionId getFeedId() {
+            return feedId;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+
+    }
+
+    public FeedRuntimeState getRuntimeState() {
+        return runtimeState;
+    }
+
+    public void setRuntimeState(FeedRuntimeState runtimeState) {
+        this.runtimeState = runtimeState;
+    }
+
+    public FeedRuntimeId getFeedRuntimeId() {
+        return feedRuntimeId;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
new file mode 100644
index 0000000..b4951ec
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
@@ -0,0 +1,234 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+public class FeedRuntimeManager {
+
+    private static Logger LOGGER = Logger.getLogger(FeedRuntimeManager.class.getName());
+
+    private final FeedConnectionId feedId;
+    private SuperFeedManager superFeedManager;
+    private final Map<FeedRuntimeId, FeedRuntime> feedRuntimes;
+    private final ExecutorService executorService;
+    private FeedMessageService messageService;
+    private SocketFactory socketFactory = new SocketFactory();
+    private final LinkedBlockingQueue<String> feedReportQueue;
+
+    public FeedRuntimeManager(FeedConnectionId feedId) {
+        this.feedId = feedId;
+        feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
+        executorService = Executors.newCachedThreadPool();
+        feedReportQueue = new LinkedBlockingQueue<String>();
+    }
+
+    public void close(boolean closeAll) throws IOException {
+        socketFactory.close();
+
+        if (messageService != null) {
+            messageService.stop();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Shut down message service s for :" + feedId);
+            }
+            messageService = null;
+        }
+        if (superFeedManager != null && superFeedManager.isLocal()) {
+            superFeedManager.stop();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Shut down super feed manager for :" + feedId);
+            }
+        }
+
+        if (closeAll) {
+            if (executorService != null) {
+                executorService.shutdownNow();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Shut down executor service for :" + feedId);
+                }
+            }
+        }
+    }
+
+    public void setSuperFeedManager(SuperFeedManager sfm) throws UnknownHostException, IOException, Exception {
+        this.superFeedManager = sfm;
+        if (sfm.isLocal()) {
+            sfm.start();
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Started Super Feed Manager for feed :" + feedId);
+        }
+        this.messageService = new FeedMessageService(feedId);
+        messageService.start();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Started Feed Message Service for feed :" + feedId);
+        }
+    }
+
+    public SuperFeedManager getSuperFeedManager() {
+        return superFeedManager;
+    }
+
+    public FeedRuntime getFeedRuntime(FeedRuntimeId runtimeId) {
+        return feedRuntimes.get(runtimeId);
+    }
+
+    public void registerFeedRuntime(FeedRuntimeId runtimeId, FeedRuntime feedRuntime) {
+        feedRuntimes.put(runtimeId, feedRuntime);
+    }
+
+    public void deregisterFeedRuntime(FeedRuntimeId runtimeId) {
+        feedRuntimes.remove(runtimeId);
+        if (feedRuntimes.isEmpty()) {
+            synchronized (this) {
+                if (feedRuntimes.isEmpty()) {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("De-registering feed");
+                    }
+                    FeedManager.INSTANCE.deregisterFeed(runtimeId.getFeedId());
+                }
+            }
+        }
+    }
+
+    public ExecutorService getExecutorService() {
+        return executorService;
+    }
+
+    public FeedMessageService getMessageService() {
+        return messageService;
+    }
+
+    public Socket createClientSocket(String host, int port) throws UnknownHostException, IOException {
+        return socketFactory.createClientSocket(host, port);
+    }
+
+    public Socket createClientSocket(String host, int port, long timeout) throws UnknownHostException, IOException {
+        Socket client = null;
+        boolean continueAttempt = true;
+        long startAttempt = System.currentTimeMillis();
+        long endAttempt = System.currentTimeMillis();
+        while (client == null && continueAttempt) {
+            try {
+                client = socketFactory.createClientSocket(host, port);
+            } catch (Exception e) {
+                endAttempt = System.currentTimeMillis();
+                if (endAttempt - startAttempt > timeout) {
+                    continueAttempt = false;
+                }
+            }
+        }
+        return client;
+    }
+
+    public ServerSocket createServerSocket(int port) throws IOException {
+        return socketFactory.createServerSocket(port);
+    }
+
+    private static class SocketFactory {
+
+        private final Map<SocketId, Socket> sockets = new HashMap<SocketId, Socket>();
+        private final List<ServerSocket> serverSockets = new ArrayList<ServerSocket>();
+
+        public Socket createClientSocket(String host, int port) throws UnknownHostException, IOException {
+            Socket socket = new Socket(host, port);
+            sockets.put(new SocketId(host, port), socket);
+            return socket;
+        }
+
+        public void close() throws IOException {
+            for (ServerSocket socket : serverSockets) {
+                socket.close();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Closed server socket :" + socket);
+                }
+            }
+
+            for (Entry<SocketId, Socket> entry : sockets.entrySet()) {
+                entry.getValue().close();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Closed client socket :" + entry.getKey());
+                }
+            }
+        }
+
+        public ServerSocket createServerSocket(int port) throws IOException {
+            ServerSocket socket = new ServerSocket(port);
+            serverSockets.add(socket);
+            return socket;
+        }
+
+        public ServerSocket createServerSocket() throws IOException {
+            ServerSocket socket = new ServerSocket(0);
+            serverSockets.add(socket);
+            return socket;
+        }
+
+        private static class SocketId {
+            private final String host;
+            private final int port;
+
+            public SocketId(String host, int port) {
+                this.host = host;
+                this.port = port;
+            }
+
+            public SocketId(int port) {
+                this.host = "127.0.0.1";
+                this.port = port;
+            }
+
+            public String getHost() {
+                return host;
+            }
+
+            public int getPort() {
+                return port;
+            }
+
+            @Override
+            public String toString() {
+                return host + "[" + port + "]";
+            }
+
+            @Override
+            public int hashCode() {
+                return toString().hashCode();
+            }
+
+            @Override
+            public boolean equals(Object o) {
+                if (!(o instanceof SocketId)) {
+                    return false;
+                }
+
+                return ((SocketId) o).getHost().equals(host) && ((SocketId) o).getPort() == port;
+            }
+
+        }
+    }
+
+    public FeedConnectionId getFeedId() {
+        return feedId;
+    }
+
+    public LinkedBlockingQueue<String> getFeedReportQueue() {
+        return feedReportQueue;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java
new file mode 100644
index 0000000..268e30a
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java
@@ -0,0 +1,46 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FeedSourceProperties {
+
+    public enum ExchangeMode {
+        PULL,
+        PUSH
+    }
+
+    public enum IngestionRuntimeType {
+        STATELESS,
+        STATEFULL
+    }
+
+    public enum ComputeRuntimeType {
+        STATELESS,
+        STATEFULL
+    }
+
+    public static class FeedSourcePropertyKeys {
+        public static final String EXCHANGE_MODE = "exchange_mode";
+        public static final String INGESTION_RUNTIME_TYPE = "ingestion_runtime_type";
+        public static final String COMPUTE_RUNTIME_TYPE = "compute_runtime_type";
+        public static final String BACKWARD_TIME_TRAVEL = "backward_time_travel";
+        public static final String COMPUTE_IDEMPOTENCE = "compute_idempotence";
+    }
+
+    private Map<String, String> sourceConfiguration;
+
+    public FeedSourceProperties(Map<String, String> sourceConfiguration) {
+        this.sourceConfiguration = sourceConfiguration;
+    }
+
+    public static class FeedSourcePropertyAccessor {
+
+        private boolean computeIdempotence;
+        private boolean backwardTimeTravel;
+        private ExchangeMode exchangeMode;
+        private IngestionRuntimeType ingestionRuntimeType;
+        private ComputeRuntimeType computeRuntimeType;
+
+    }
+}
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
new file mode 100644
index 0000000..c1cddb7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
@@ -0,0 +1,229 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
+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;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+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;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedUtil {
+
+    private static Logger LOGGER = Logger.getLogger(FeedUtil.class.getName());
+
+    public static boolean isFeedActive(FeedActivity feedActivity) {
+        return (feedActivity != null && !(feedActivity.getActivityType().equals(FeedActivityType.FEED_FAILURE) || feedActivity
+                .getActivityType().equals(FeedActivityType.FEED_END)));
+    }
+
+    public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
+            FeedConnectionId feedConnectionId, FeedPolicy feedPolicy) {
+        JobSpecification altered = null;
+        altered = new JobSpecification();
+        Map<OperatorDescriptorId, IOperatorDescriptor> operatorMap = spec.getOperatorMap();
+
+        // copy operators
+        Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<OperatorDescriptorId, OperatorDescriptorId>();
+        for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
+            IOperatorDescriptor opDesc = entry.getValue();
+            if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+                FeedIntakeOperatorDescriptor orig = (FeedIntakeOperatorDescriptor) opDesc;
+                FeedIntakeOperatorDescriptor fiop = new FeedIntakeOperatorDescriptor(altered, orig.getFeedId(),
+                        orig.getAdapterFactory(), (ARecordType) orig.getAtype(), orig.getRecordDescriptor(),
+                        orig.getFeedPolicy());
+                oldNewOID.put(opDesc.getOperatorId(), fiop.getOperatorId());
+            } else if (opDesc instanceof AsterixLSMTreeInsertDeleteOperatorDescriptor) {
+                FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
+                        feedPolicy, FeedRuntimeType.STORAGE);
+                oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+            } else {
+                FeedRuntimeType runtimeType = null;
+                if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
+                    IPushRuntimeFactory runtimeFactory = ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline()
+                            .getRuntimeFactories()[0];
+                    if (runtimeFactory instanceof AssignRuntimeFactory) {
+                        runtimeType = FeedRuntimeType.COMPUTE;
+                    } else if (runtimeFactory instanceof StreamProjectRuntimeFactory) {
+                        runtimeType = FeedRuntimeType.COMMIT;
+                    }
+                }
+                FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
+                        feedPolicy, runtimeType);
+
+                oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+            }
+        }
+
+        // copy connectors
+        Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorMapping = new HashMap<ConnectorDescriptorId, ConnectorDescriptorId>();
+        for (Entry<ConnectorDescriptorId, IConnectorDescriptor> entry : spec.getConnectorMap().entrySet()) {
+            IConnectorDescriptor connDesc = entry.getValue();
+            ConnectorDescriptorId newConnId = altered.createConnectorDescriptor(connDesc);
+            connectorMapping.put(entry.getKey(), newConnId);
+        }
+
+        // make connections between operators
+        for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> entry : spec
+                .getConnectorOperatorMap().entrySet()) {
+            IConnectorDescriptor connDesc = altered.getConnectorMap().get(connectorMapping.get(entry.getKey()));
+            Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
+            Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
+
+            IOperatorDescriptor leftOpDesc = altered.getOperatorMap().get(
+                    oldNewOID.get(leftOp.getLeft().getOperatorId()));
+            IOperatorDescriptor rightOpDesc = altered.getOperatorMap().get(
+                    oldNewOID.get(rightOp.getLeft().getOperatorId()));
+
+            altered.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
+        }
+
+        // prepare for setting partition constraints
+        Map<OperatorDescriptorId, Map<Integer, String>> operatorLocations = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
+        Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<OperatorDescriptorId, Integer>();
+
+        for (Constraint constraint : spec.getUserConstraints()) {
+            LValueConstraintExpression lexpr = constraint.getLValue();
+            ConstraintExpression cexpr = constraint.getRValue();
+            OperatorDescriptorId opId;
+            switch (lexpr.getTag()) {
+                case PARTITION_COUNT:
+                    opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+                    operatorCounts.put(opId, (int) ((ConstantExpression) cexpr).getValue());
+                    break;
+                case PARTITION_LOCATION:
+                    opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+                    IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(opId));
+                    Map<Integer, String> locations = operatorLocations.get(opDesc.getOperatorId());
+                    if (locations == null) {
+                        locations = new HashMap<Integer, String>();
+                        operatorLocations.put(opDesc.getOperatorId(), locations);
+                    }
+                    String location = (String) ((ConstantExpression) cexpr).getValue();
+                    int partition = ((PartitionLocationExpression) lexpr).getPartition();
+                    locations.put(partition, location);
+                    break;
+            }
+        }
+
+        // set absolute location constraints
+        for (Entry<OperatorDescriptorId, Map<Integer, String>> entry : operatorLocations.entrySet()) {
+            IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
+            String[] locations = new String[entry.getValue().size()];
+            for (Entry<Integer, String> e : entry.getValue().entrySet()) {
+                locations[e.getKey()] = e.getValue();
+            }
+            PartitionConstraintHelper.addAbsoluteLocationConstraint(altered, opDesc, locations);
+        }
+
+        // set count constraints
+        for (Entry<OperatorDescriptorId, Integer> entry : operatorCounts.entrySet()) {
+            IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
+            if (!operatorLocations.keySet().contains(entry.getKey())) {
+                PartitionConstraintHelper.addPartitionCountConstraint(altered, opDesc, entry.getValue());
+            }
+        }
+
+        // useConnectorSchedulingPolicy
+        altered.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
+
+        // connectorAssignmentPolicy
+        altered.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
+
+        // roots
+        for (OperatorDescriptorId root : spec.getRoots()) {
+            altered.addRoot(altered.getOperatorMap().get(oldNewOID.get(root)));
+        }
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("New Job Spec:" + altered);
+        }
+
+        return altered;
+
+    }
+
+    public static Pair<IAdapterFactory, ARecordType> getFeedFactoryAndOutput(Feed feed,
+            MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+
+        String adapterName = null;
+        DatasourceAdapter adapterEntity = null;
+        String adapterFactoryClassname = null;
+        IAdapterFactory adapterFactory = null;
+        ARecordType adapterOutputType = null;
+        Pair<IAdapterFactory, ARecordType> feedProps = null;
+        try {
+            adapterName = feed.getAdaptorName();
+            adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                    adapterName);
+            if (adapterEntity != null) {
+                adapterFactoryClassname = adapterEntity.getClassname();
+                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+            } else {
+                adapterFactoryClassname = AqlMetadataProvider.adapterFactoryMapping.get(adapterName);
+                if (adapterFactoryClassname != null) {
+                } else {
+                    adapterFactoryClassname = adapterName;
+                }
+                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+            }
+
+            Map<String, String> configuration = feed.getAdaptorConfiguration();
+
+            switch (adapterFactory.getAdapterType()) {
+                case TYPED:
+                    adapterOutputType = ((ITypedAdapterFactory) adapterFactory).getAdapterOutputType();
+                    ((ITypedAdapterFactory) adapterFactory).configure(configuration);
+                    break;
+                case GENERIC:
+                    String outputTypeName = configuration.get("output-type-name");
+                    adapterOutputType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            feed.getDataverseName(), outputTypeName).getDatatype();
+                    ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType);
+                    break;
+                default:
+                    throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
+            }
+
+            feedProps = Pair.of(adapterFactory, adapterOutputType);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new AlgebricksException("unable to create adapter  " + e);
+        }
+        return feedProps;
+    }
+}
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
new file mode 100644
index 0000000..0d1f955
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+public interface IAdapterExecutor {
+
+    /**
+     * @throws Exception
+     */
+    public void start() throws Exception;
+
+    /**
+     * @throws Exception
+     */
+    public void stop() throws Exception;
+
+    /**
+     * @return
+     */
+    public FeedConnectionId getFeedId();
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java
new file mode 100644
index 0000000..da4f131
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
+ * Acts as a marker interface indicating that the implementation provides functionality
+ * for creating an adapter.
+ */
+public interface IAdapterFactory extends Serializable {
+
+    /**
+     * A 'GENERIC' adapter can be configured to return a given datatype.
+     * A 'TYPED' adapter returns records with a pre-defined datatype.
+     */
+    public enum AdapterType {
+        GENERIC,
+        TYPED
+    }
+
+    public enum SupportedOperation {
+        READ,
+        WRITE,
+        READ_WRITE
+    }
+
+    /**
+     * Returns the type of adapter indicating if the adapter can be used for
+     * reading from an external data source or writing to an external data
+     * source or can be used for both purposes.
+     * 
+     * @see SupportedOperation
+     * @return
+     */
+    public SupportedOperation getSupportedOperations();
+
+    /**
+     * Returns the display name corresponding to the Adapter type that is created by the factory.
+     * 
+     * @return the display name
+     */
+    public String getName();
+
+    /**
+     * Returns the type of the adapter (GENERIC or TYPED)
+     * 
+     * @return
+     */
+    public AdapterType getAdapterType();
+
+    /**
+     * Returns a list of partition constraints. A partition constraint can be a
+     * requirement to execute at a particular location or could be cardinality
+     * constraints indicating the number of instances that need to run in
+     * parallel. example, a IDatasourceAdapter implementation written for data
+     * residing on the local file system of a node cannot run on any other node
+     * and thus has a location partition constraint. The location partition
+     * constraint can be expressed as a node IP address or a node controller id.
+     * In the former case, the IP address is translated to a node controller id
+     * running on the node with the given IP address.
+     */
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
+
+    /**
+     * Creates an instance of IDatasourceAdapter.
+     * 
+     * @param HyracksTaskContext
+     * @return An instance of IDatasourceAdapter.
+     * @throws Exception
+     */
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception;
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
new file mode 100644
index 0000000..d0f8cdd
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+/**
+ * A super interface implemented by a data source adapter. An adapter can be a
+ * pull based or push based. This interface provides all common APIs that need
+ * to be implemented by each adapter irrespective of the the kind of
+ * adapter(pull or push).
+ */
+public interface IDatasourceAdapter extends Serializable {
+
+    /**
+     * An adapter can be used to read from an external data source and may also
+     * allow writing to the external data source. This enum type indicates the
+     * kind of operations supported by the adapter.
+     * 
+     * @caller Compiler uses this method to assert the validity of an operation
+     *         on an external dataset. The type of adapter associated with an
+     *         external dataset determines the set of valid operations allowed
+     *         on the dataset.
+     */
+   
+
+    /**
+     * Triggers the adapter to begin ingesting data from the external source.
+     * 
+     * @param partition
+     *            The adapter could be running with a degree of parallelism.
+     *            partition corresponds to the i'th parallel instance.
+     * @param writer
+     *            The instance of frame writer that is used by the adapter to
+     *            write frame to. Adapter packs the fetched bytes (from external source),
+     *            packs them into frames and forwards the frames to an upstream receiving
+     *            operator using the instance of IFrameWriter.
+     * @throws Exception
+     */
+    public void start(int partition, IFrameWriter writer) throws Exception;
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
similarity index 63%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
copy to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
index 00b8530..f525420 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
@@ -12,16 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
+package edu.uci.ics.asterix.metadata.feeds;
 
 /**
- * Implemented by datasource adapter that has a fixed output type.
- * Example @see {PullBasedTwitterAdapter}
+ * Interface implemented by an adapter that can be controlled or managed by external
+ * commands (stop,alter)
  */
-public interface ITypedDatasourceAdapter extends IDatasourceAdapter {
+public interface IFeedAdapter extends IDatasourceAdapter {
 
-    public ARecordType getAdapterOutputType();
+    /**
+     * Discontinue the ingestion of data and end the feed.
+     * 
+     * @throws Exception
+     */
+    public void stop() throws Exception;
 
 }
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
new file mode 100644
index 0000000..ac33966
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
@@ -0,0 +1,69 @@
+/*
+ * 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.IOException;
+import java.util.concurrent.ExecutorService;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+/**
+ * Handle (de)registration of feeds for delivery of control messages.
+ */
+public interface IFeedManager {
+
+    /**
+     * @param feedId
+     * @return
+     */
+    public ExecutorService getFeedExecutorService(FeedConnectionId feedId);
+
+    /**
+     * @param feedRuntime
+     * @throws Exception
+     */
+    public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception;
+
+    /**
+     * @param feedRuntimeId
+     */
+    public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId);
+
+    /**
+     * @param feedRuntimeId
+     * @return
+     */
+    public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId);
+
+    /**
+     * @param feedId
+     * @param sfm
+     * @throws Exception
+     */
+    public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception;
+
+    /**
+     * @param feedId
+     * @return
+     */
+    public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId);
+
+    /**
+     * @param feedId
+     * @throws IOException
+     */
+    void deregisterFeed(FeedConnectionId feedId) throws IOException;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
similarity index 89%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
index dcef2c8..0241e5b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
@@ -12,15 +12,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.feeds;
 
 import java.io.Serializable;
 
 public interface IFeedMessage extends Serializable {
 
     public enum MessageType {
-        STOP,
-        ALTER,
+        END,
+        SUPER_FEED_MANAGER_ELECT
     }
 
     public MessageType getMessageType();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
new file mode 100644
index 0000000..ac0c711
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+
+public interface IGenericAdapterFactory extends IAdapterFactory {
+
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
similarity index 74%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
index 00b8530..6faa44b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
@@ -12,16 +12,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.external.dataset.adapter;
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
 
 import edu.uci.ics.asterix.om.types.ARecordType;
 
-/**
- * Implemented by datasource adapter that has a fixed output type.
- * Example @see {PullBasedTwitterAdapter}
- */
-public interface ITypedDatasourceAdapter extends IDatasourceAdapter {
+public interface ITypedAdapterFactory extends IAdapterFactory {
 
     public ARecordType getAdapterOutputType();
 
+    public void configure(Map<String, String> configuration) throws Exception;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java
new file mode 100644
index 0000000..37d49b4
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+public class IngestionRuntime extends FeedRuntime {
+
+    private AdapterRuntimeManager adapterRuntimeManager;
+
+    public IngestionRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType,
+            AdapterRuntimeManager adaptorRuntimeManager) {
+        super(feedId, partition, feedRuntimeType);
+        this.adapterRuntimeManager = adaptorRuntimeManager;
+    }
+
+    public AdapterRuntimeManager getAdapterRuntimeManager() {
+        return adapterRuntimeManager;
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java
new file mode 100644
index 0000000..20a6b0e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java
@@ -0,0 +1,165 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class MessageListener {
+
+    private static final Logger LOGGER = Logger.getLogger(MessageListener.class.getName());
+
+    private int port;
+    private final LinkedBlockingQueue<String> outbox;
+
+    private ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+    private MessageListenerServer listenerServer;
+
+    public MessageListener(int port, LinkedBlockingQueue<String> outbox) {
+        this.port = port;
+        this.outbox = outbox;
+    }
+
+    public void stop() {
+        listenerServer.stop();
+        System.out.println("STOPPED MESSAGE RECEIVING SERVICE AT " + port);
+        if (!executorService.isShutdown()) {
+            executorService.shutdownNow();
+        }
+
+    }
+
+    public void start() throws IOException {
+        System.out.println("STARTING MESSAGE RECEIVING SERVICE AT " + port);
+        listenerServer = new MessageListenerServer(port, outbox);
+        executorService.execute(listenerServer);
+    }
+
+    private static class MessageListenerServer implements Runnable {
+
+        private final int port;
+        private final LinkedBlockingQueue<String> outbox;
+        private ServerSocket server;
+
+        public MessageListenerServer(int port, LinkedBlockingQueue<String> outbox) {
+            this.port = port;
+            this.outbox = outbox;
+        }
+
+        public void stop() {
+            try {
+                server.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public void run() {
+            char EOL = (char) "\n".getBytes()[0];
+            Socket client = null;
+            try {
+                server = new ServerSocket(port);
+                client = server.accept();
+                InputStream in = client.getInputStream();
+                CharBuffer buffer = CharBuffer.allocate(5000);
+                char ch;
+                while (true) {
+                    ch = (char) in.read();
+                    if (((int) ch) == -1) {
+                        break;
+                    }
+                    while (ch != EOL) {
+                        buffer.put(ch);
+                        ch = (char) in.read();
+                    }
+                    buffer.flip();
+                    String s = new String(buffer.array());
+                    synchronized (outbox) {
+                        outbox.add(s + "\n");
+                    }
+                    buffer.position(0);
+                    buffer.limit(5000);
+                }
+
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Unable to start Message listener" + server);
+                }
+            } finally {
+                if (server != null) {
+                    try {
+                        server.close();
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+        }
+
+    }
+
+    private static class MessageParser implements Runnable {
+
+        private Socket client;
+        private IMessageAnalyzer messageAnalyzer;
+        private static final char EOL = (char) "\n".getBytes()[0];
+
+        public MessageParser(Socket client, IMessageAnalyzer messageAnalyzer) {
+            this.client = client;
+            this.messageAnalyzer = messageAnalyzer;
+        }
+
+        @Override
+        public void run() {
+            CharBuffer buffer = CharBuffer.allocate(5000);
+            char ch;
+            try {
+                InputStream in = client.getInputStream();
+                while (true) {
+                    ch = (char) in.read();
+                    if (((int) ch) == -1) {
+                        break;
+                    }
+                    while (ch != EOL) {
+                        buffer.put(ch);
+                        ch = (char) in.read();
+                    }
+                    buffer.flip();
+                    String s = new String(buffer.array());
+                    synchronized (messageAnalyzer) {
+                        messageAnalyzer.getMessageQueue().add(s + "\n");
+                    }
+                    buffer.position(0);
+                    buffer.limit(5000);
+                }
+            } catch (IOException ioe) {
+                ioe.printStackTrace();
+            } finally {
+                try {
+                    client.close();
+                } catch (IOException ioe) {
+                    // do nothing
+                }
+            }
+        }
+    }
+
+    public static interface IMessageAnalyzer {
+
+        /**
+         * @return
+         */
+        public LinkedBlockingQueue<String> getMessageQueue();
+
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java
new file mode 100644
index 0000000..822c638
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java
@@ -0,0 +1,165 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class RemoteSocketMessageListener {
+
+    private static final Logger LOGGER = Logger.getLogger(RemoteSocketMessageListener.class.getName());
+
+    private final String host;
+    private final int port;
+    private final LinkedBlockingQueue<String> outbox;
+
+    private ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+    private RemoteMessageListenerServer listenerServer;
+
+    public RemoteSocketMessageListener(String host, int port, LinkedBlockingQueue<String> outbox) {
+        this.host = host;
+        this.port = port;
+        this.outbox = outbox;
+    }
+
+    public void stop() {
+        if (!executorService.isShutdown()) {
+            executorService.shutdownNow();
+        }
+        listenerServer.stop();
+
+    }
+
+    public void start() throws IOException {
+        listenerServer = new RemoteMessageListenerServer(host, port, outbox);
+        executorService.execute(listenerServer);
+    }
+
+    private static class RemoteMessageListenerServer implements Runnable {
+
+        private final String host;
+        private final int port;
+        private final LinkedBlockingQueue<String> outbox;
+        private Socket client;
+
+        public RemoteMessageListenerServer(String host, int port, LinkedBlockingQueue<String> outbox) {
+            this.host = host;
+            this.port = port;
+            this.outbox = outbox;
+        }
+
+        public void stop() {
+            try {
+                client.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public void run() {
+            char EOL = (char) "\n".getBytes()[0];
+            Socket client = null;
+            try {
+                client = new Socket(host, port);
+                InputStream in = client.getInputStream();
+                CharBuffer buffer = CharBuffer.allocate(5000);
+                char ch;
+                while (true) {
+                    ch = (char) in.read();
+                    if (((int) ch) == -1) {
+                        break;
+                    }
+                    while (ch != EOL) {
+                        buffer.put(ch);
+                        ch = (char) in.read();
+                    }
+                    buffer.flip();
+                    String s = new String(buffer.array());
+                    synchronized (outbox) {
+                        outbox.add(s + "\n");
+                    }
+                    buffer.position(0);
+                    buffer.limit(5000);
+                }
+
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Unable to start Remote Message listener" + client);
+                }
+            } finally {
+                if (client != null && !client.isClosed()) {
+                    try {
+                        client.close();
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+        }
+
+    }
+
+    private static class MessageParser implements Runnable {
+
+        private Socket client;
+        private IMessageAnalyzer messageAnalyzer;
+        private static final char EOL = (char) "\n".getBytes()[0];
+
+        public MessageParser(Socket client, IMessageAnalyzer messageAnalyzer) {
+            this.client = client;
+            this.messageAnalyzer = messageAnalyzer;
+        }
+
+        @Override
+        public void run() {
+            CharBuffer buffer = CharBuffer.allocate(5000);
+            char ch;
+            try {
+                InputStream in = client.getInputStream();
+                while (true) {
+                    ch = (char) in.read();
+                    if (((int) ch) == -1) {
+                        break;
+                    }
+                    while (ch != EOL) {
+                        buffer.put(ch);
+                        ch = (char) in.read();
+                    }
+                    buffer.flip();
+                    String s = new String(buffer.array());
+                    synchronized (messageAnalyzer) {
+                        messageAnalyzer.getMessageQueue().add(s + "\n");
+                    }
+                    buffer.position(0);
+                    buffer.limit(5000);
+                }
+            } catch (IOException ioe) {
+                ioe.printStackTrace();
+            } finally {
+                try {
+                    client.close();
+                } catch (IOException ioe) {
+                    // do nothing
+                }
+            }
+        }
+    }
+
+    public static interface IMessageAnalyzer {
+
+        /**
+         * @return
+         */
+        public LinkedBlockingQueue<String> getMessageQueue();
+
+    }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
new file mode 100644
index 0000000..a45456b6
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
@@ -0,0 +1,443 @@
+/*
+ * 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.IOException;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener.IMessageAnalyzer;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+/**
+ * The feed operators running in an NC report their health (statistics) to the local Feed Manager.
+ * A feed thus has a Feed Manager per NC. From amongst the Feed Maanger, a SuperFeedManager is chosen (randomly)
+ * The SuperFeedManager collects reports from the FeedMaangers and has the global cluster view in terms of
+ * how different feed operators running in a distributed fashion are performing.
+ */
+public class SuperFeedManager {
+
+    private static final Logger LOGGER = Logger.getLogger(SuperFeedManager.class.getName());
+
+    /**
+     * IP Address or DNS name of the host where Super Feed Manager is running.
+     */
+    private String host;
+
+    private AtomicInteger availablePort; // starting value is fixed
+
+    /**
+     * The port at which the SuperFeedManager listens for connections by other Feed Managers.
+     */
+    private final int feedReportPort; // fixed
+
+    /**
+     * The port at which the SuperFeedManager listens for connections by clients that wish
+     * to subscribe to the feed health reports.E.g. feed management console.
+     */
+    private final int feedReportSubscribePort; // fixed
+
+    /**
+     * The Id of Node Controller
+     */
+    private final String nodeId;
+
+    /**
+     * A unique identifier for the feed instance. A feed instance represents the flow of data
+     * from a feed to a dataset.
+     **/
+    private final FeedConnectionId feedConnectionId;
+
+    /**
+     * Set to true of the Super Feed Manager is local to the NC.
+     **/
+    private boolean isLocal = false;
+
+    private FeedReportDestinationSocketProvider sfmService;
+
+    private SuperFeedReportSubscriptionService subscriptionService;
+
+    private LinkedBlockingQueue<String> feedReportInbox; ///
+
+    private boolean started = false;
+
+    public static final int PORT_RANGE_ASSIGNED = 10;
+
+    public enum FeedReportMessageType {
+        CONGESTION,
+        THROUGHPUT
+    }
+
+    public SuperFeedManager(FeedConnectionId feedId, String host, String nodeId, int port) throws Exception {
+        this.feedConnectionId = feedId;
+        this.nodeId = nodeId;
+        this.feedReportPort = port;
+        this.feedReportSubscribePort = port + 1;
+        this.availablePort = new AtomicInteger(feedReportSubscribePort + 1);
+        this.host = host;
+        this.feedReportInbox = new LinkedBlockingQueue<String>();
+    }
+
+    public int getPort() {
+        return feedReportPort;
+    }
+
+    public String getHost() throws Exception {
+        return host;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public FeedConnectionId getFeedConnectionId() {
+        return feedConnectionId;
+    }
+
+    public boolean isLocal() {
+        return isLocal;
+    }
+
+    public void setLocal(boolean isLocal) {
+        this.isLocal = isLocal;
+    }
+
+    public void start() throws IOException {
+        if (sfmService == null) {
+            ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedConnectionId);
+            sfmService = new FeedReportDestinationSocketProvider(feedReportPort, feedReportInbox, feedConnectionId,
+                    availablePort);
+            executorService.execute(sfmService);
+            subscriptionService = new SuperFeedReportSubscriptionService(feedConnectionId, feedReportSubscribePort,
+                    sfmService.getMesgAnalyzer(), availablePort);
+            executorService.execute(subscriptionService);
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Started super feed manager! " + this);
+        }
+        started = true;
+    }
+
+    public void stop() throws IOException {
+        sfmService.stop();
+        subscriptionService.stop();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Stopped super feed manager! " + this);
+        }
+        started = false;
+    }
+
+    public boolean isStarted() {
+        return started;
+    }
+
+    @Override
+    public String toString() {
+        return feedConnectionId + "[" + nodeId + "(" + host + ")" + ":" + feedReportPort + "]"
+                + (isLocal ? started ? "Started " : "Not Started" : " Remote ");
+    }
+
+    public AtomicInteger getAvailablePort() {
+        return availablePort;
+    }
+
+    private static class SuperFeedReportSubscriptionService implements Runnable {
+
+        private final FeedConnectionId feedId;
+        private ServerSocket serverFeedSubscribe;
+        private AtomicInteger subscriptionPort;
+        private boolean active = true;
+        private String EOM = "\n";
+        private final FeedReportProvider reportProvider;
+        private final List<FeedDataProviderService> dataProviders = new ArrayList<FeedDataProviderService>();
+
+        public SuperFeedReportSubscriptionService(FeedConnectionId feedId, int port, FeedReportProvider reportProvider,
+                AtomicInteger nextPort) throws IOException {
+            this.feedId = feedId;
+            serverFeedSubscribe = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+            this.subscriptionPort = nextPort;
+            this.reportProvider = reportProvider;
+        }
+
+        public void stop() {
+            active = false;
+            for (FeedDataProviderService dataProviderService : dataProviders) {
+                dataProviderService.stop();
+            }
+        }
+
+        @Override
+        public void run() {
+            while (active) {
+                try {
+                    Socket client = serverFeedSubscribe.accept();
+                    OutputStream os = client.getOutputStream();
+                    int port = subscriptionPort.incrementAndGet();
+                    LinkedBlockingQueue<String> reportInbox = new LinkedBlockingQueue<String>();
+                    reportProvider.registerSubsription(reportInbox);
+                    FeedDataProviderService dataProviderService = new FeedDataProviderService(feedId, port, reportInbox);
+                    dataProviders.add(dataProviderService);
+                    FeedManager.INSTANCE.getFeedRuntimeManager(feedId).getExecutorService()
+                            .execute(dataProviderService);
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Recevied subscription request for feed :" + feedId
+                                + " Subscripton available at port " + subscriptionPort);
+                    }
+                    os.write((port + EOM).getBytes());
+                    os.flush();
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private static class FeedDataProviderService implements Runnable {
+
+        private final FeedConnectionId feedId;
+        private int subscriptionPort;
+        private ServerSocket dataProviderSocket;
+        private LinkedBlockingQueue<String> inbox;
+        private boolean active = true;
+        private String EOM = "\n";
+
+        public FeedDataProviderService(FeedConnectionId feedId, int port, LinkedBlockingQueue<String> inbox)
+                throws IOException {
+            this.feedId = feedId;
+            this.subscriptionPort = port;
+            this.inbox = inbox;
+            dataProviderSocket = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+        }
+
+        @Override
+        public void run() {
+            try {
+                Socket client = dataProviderSocket.accept();
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Subscriber to " + feedId + " data connected");
+                }
+                OutputStream os = client.getOutputStream();
+                while (active) {
+                    String message = inbox.take();
+                    os.write((message + EOM).getBytes());
+                }
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Unsubscribed from " + feedId + " disconnected");
+                }
+            } catch (IOException e) {
+                e.printStackTrace();
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        }
+
+        public void stop() {
+            active = false;
+        }
+
+        @Override
+        public String toString() {
+            return "DATA_PROVIDER_" + feedId + "[" + subscriptionPort + "]";
+        }
+
+    }
+
+    private static class FeedReportDestinationSocketProvider implements Runnable {
+
+        private static final String EOM = "\n";
+
+        private AtomicInteger nextPort;
+        private final ServerSocket feedReportSocket;
+        private final LinkedBlockingQueue<String> inbox;
+        private final List<MessageListener> messageListeners;
+        private final FeedReportProvider mesgAnalyzer;
+        private final FeedConnectionId feedId;
+        private boolean process = true;
+
+        public FeedReportDestinationSocketProvider(int port, LinkedBlockingQueue<String> inbox,
+                FeedConnectionId feedId, AtomicInteger availablePort) throws IOException {
+            FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+            this.feedReportSocket = runtimeManager.createServerSocket(port);
+            this.nextPort = availablePort;
+            this.inbox = inbox;
+            this.feedId = feedId;
+            this.messageListeners = new ArrayList<MessageListener>();
+            this.mesgAnalyzer = new FeedReportProvider(inbox, feedId);
+            FeedManager.INSTANCE.getFeedExecutorService(feedId).execute(mesgAnalyzer);
+        }
+
+        public void stop() {
+            process = false;
+            if (feedReportSocket != null) {
+                try {
+                    feedReportSocket.close();
+                    process = false;
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+            for (MessageListener listener : messageListeners) {
+                listener.stop();
+            }
+            mesgAnalyzer.stop();
+        }
+
+        @Override
+        public void run() {
+            Socket client = null;
+            while (process) {
+                try {
+                    client = feedReportSocket.accept();
+                    int port = nextPort.incrementAndGet();
+                    /**
+                     * MessageListener provides the functionality of listening at a port for messages
+                     * and delivering each received message to an input queue (inbox).
+                     */
+                    MessageListener listener = new MessageListener(port, inbox);
+                    listener.start();
+                    synchronized (messageListeners) {
+                        messageListeners.add(listener);
+                    }
+                    OutputStream os = client.getOutputStream();
+                    os.write((port + EOM).getBytes());
+                    os.flush();
+                } catch (IOException e) {
+                    if (process == false) {
+                        break;
+                    }
+                } finally {
+                    if (client != null) {
+                        try {
+                            client.close();
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                }
+            }
+        }
+
+        public FeedReportProvider getMesgAnalyzer() {
+            return mesgAnalyzer;
+        }
+
+    }
+
+    /**
+     * The report messages sent by the feed operators are sent to the FeedReportProvider.
+     * FeedReportMessageAnalyzer is responsible for distributing the messages to the subscribers.
+     * The Feed Management Console is an example of a subscriber.
+     */
+    private static class FeedReportProvider implements Runnable {
+
+        private final LinkedBlockingQueue<String> inbox;
+        private final FeedConnectionId feedId;
+        private boolean process = true;
+        private final List<LinkedBlockingQueue<String>> subscriptionQueues;
+        private final Map<String, String> ingestionThroughputs;
+
+        public FeedReportProvider(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId)
+                throws UnknownHostException, IOException {
+            this.inbox = inbox;
+            this.feedId = feedId;
+            this.subscriptionQueues = new ArrayList<LinkedBlockingQueue<String>>();
+            this.ingestionThroughputs = new HashMap<String, String>();
+        }
+
+        public void stop() {
+            process = false;
+        }
+
+        public void registerSubsription(LinkedBlockingQueue<String> subscriptionQueue) {
+            subscriptionQueues.add(subscriptionQueue);
+        }
+
+        public void deregisterSubsription(LinkedBlockingQueue<String> subscriptionQueue) {
+            subscriptionQueues.remove(subscriptionQueue);
+        }
+
+        public void run() {
+            StringBuilder finalMessage = new StringBuilder();
+            FeedReport report = new FeedReport();
+            while (process) {
+                try {
+                    String message = inbox.take();
+                    report.reset(message);
+                    FeedReportMessageType mesgType = report.getReportType();
+                    switch (mesgType) {
+                        case THROUGHPUT:
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.warning("Feed Health Report " + message);
+                            }
+                            String[] msgComponents = message.split("\\|");
+                            String partition = msgComponents[3];
+                            String tput = msgComponents[4];
+                            String timestamp = msgComponents[6];
+
+                            boolean dispatchReport = true;
+                            if (ingestionThroughputs.get(partition) == null) {
+                                ingestionThroughputs.put(partition, tput);
+                                dispatchReport = false;
+                            } else {
+                                for (int i = 0; i < ingestionThroughputs.size(); i++) {
+                                    String tp = ingestionThroughputs.get(i + "");
+                                    if (tp != null) {
+                                        ingestionThroughputs.put(i + "", null);
+                                        finalMessage.append(tp + "|");
+                                    } else {
+                                        dispatchReport = false;
+                                        break;
+                                    }
+                                }
+                                ingestionThroughputs.put(partition, tput);
+                            }
+
+                            if (dispatchReport) {
+                                String dispatchedReport = finalMessage.toString();
+                                if (LOGGER.isLoggable(Level.INFO)) {
+                                    LOGGER.info("Dispatched report " + dispatchedReport);
+                                }
+                                for (LinkedBlockingQueue<String> q : subscriptionQueues) {
+                                    q.add(dispatchedReport);
+                                }
+                            }
+                            finalMessage.delete(0, finalMessage.length());
+                            break;
+                        case CONGESTION:
+                            // congestionInbox.add(report);
+                            break;
+                    }
+                } catch (InterruptedException e) {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("Unable to process messages " + e.getMessage() + " for feed " + feedId);
+                    }
+                }
+            }
+        }
+
+    }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java
new file mode 100644
index 0000000..6bf5be3
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 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.functions;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.om.functions.AsterixExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+
+public class AsterixExternalScalarFunctionInfo extends AsterixExternalFunctionInfo {
+
+    private static final long serialVersionUID = 1L;
+
+    public AsterixExternalScalarFunctionInfo(String namespace, AsterixFunction asterixFunction, IAType returnType,
+            String body, String language, List<IAType> argumentTypes, IResultTypeComputer rtc) {
+        super(namespace, asterixFunction, FunctionKind.SCALAR, argumentTypes, returnType, rtc, body, language);
+    }
+
+    public AsterixExternalScalarFunctionInfo() {
+        super();
+    }
+
+    
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
new file mode 100644
index 0000000..f699c41
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -0,0 +1,221 @@
+/*
+ * Copyright 2009-2012 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.functions;
+
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+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.Datatype;
+import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.ADoubleTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AFloatTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AInt32TypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AStringTypeComputer;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+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.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class ExternalFunctionCompilerUtil implements Serializable {
+
+    private  static Pattern orderedListPattern = Pattern.compile("\\[*\\]");
+    private  static Pattern unorderedListPattern = Pattern.compile("[{{*}}]");
+
+  
+    public static IFunctionInfo getExternalFunctionInfo(MetadataTransactionContext txnCtx, Function function)
+            throws MetadataException {
+
+        String functionKind = function.getKind();
+        IFunctionInfo finfo = null;
+        if (FunctionKind.SCALAR.toString().equalsIgnoreCase(functionKind)) {
+            finfo = getScalarFunctionInfo(txnCtx, function);
+        } else if (FunctionKind.AGGREGATE.toString().equalsIgnoreCase(functionKind)) {
+            finfo = getAggregateFunctionInfo(txnCtx, function);
+        } else if (FunctionKind.STATEFUL.toString().equalsIgnoreCase(functionKind)) {
+            finfo = getStatefulFunctionInfo(txnCtx, function);
+        } else if (FunctionKind.UNNEST.toString().equalsIgnoreCase(functionKind)) {
+            finfo = getUnnestFunctionInfo(txnCtx, function);
+        }
+        return finfo;
+    }
+
+    private static IFunctionInfo getScalarFunctionInfo(MetadataTransactionContext txnCtx, Function function)
+            throws MetadataException {
+        FunctionIdentifier fid = new FunctionIdentifier(function.getDataverseName(), function.getName(),
+                function.getArity());
+        IResultTypeComputer typeComputer = getResultTypeComputer(txnCtx, function);
+        List<IAType> arguments = new ArrayList<IAType>();
+        IAType returnType = null;
+        List<String> paramTypes = function.getParams();
+        for (String paramType : paramTypes) {
+            arguments.add(getTypeInfo(paramType, txnCtx, function));
+        }
+
+        returnType = getTypeInfo(function.getReturnType(), txnCtx, function);
+
+        AsterixExternalScalarFunctionInfo scalarFunctionInfo = new AsterixExternalScalarFunctionInfo(
+                fid.getNamespace(), new AsterixFunction(fid.getName(), fid.getArity()), returnType,
+                function.getFunctionBody(), function.getLanguage(), arguments, typeComputer);
+        return scalarFunctionInfo;
+    }
+
+    private static IAType getTypeInfo(String paramType, MetadataTransactionContext txnCtx, Function function)
+            throws MetadataException {
+        if (paramType.equalsIgnoreCase(BuiltinType.AINT32.getDisplayName())) {
+            return (BuiltinType.AINT32);
+        } else if (paramType.equalsIgnoreCase(BuiltinType.AFLOAT.getDisplayName())) {
+            return (BuiltinType.AFLOAT);
+        } else if (paramType.equalsIgnoreCase(BuiltinType.ASTRING.getDisplayName())) {
+            return (BuiltinType.ASTRING);
+        } else if (paramType.equalsIgnoreCase(BuiltinType.ADOUBLE.getDisplayName())) {
+            return (BuiltinType.ADOUBLE);
+        } else {
+            IAType collection = getCollectionType(paramType, txnCtx, function);
+            if (collection != null) {
+                return collection;
+            } else {
+                Datatype datatype;
+                datatype = MetadataManager.INSTANCE.getDatatype(txnCtx, function.getDataverseName(), paramType);
+                if (datatype == null) {
+                    throw new MetadataException(" Type " + paramType + " not defined");
+                }
+                return (datatype.getDatatype());
+            }
+        }
+    }
+
+    private static IAType getCollectionType(String paramType, MetadataTransactionContext txnCtx, Function function)
+            throws MetadataException {
+
+        Matcher matcher = orderedListPattern.matcher(paramType);
+        if (matcher.find()) {
+            String subType = paramType.substring(paramType.indexOf('[') + 1, paramType.lastIndexOf(']'));
+            return new AOrderedListType(getTypeInfo(subType, txnCtx, function), "AOrderedList");
+        } else {
+            matcher = unorderedListPattern.matcher(paramType);
+            if (matcher.find()) {
+                String subType = paramType.substring(paramType.indexOf("{{") + 2, paramType.lastIndexOf("}}"));
+                return new AUnorderedListType(getTypeInfo(subType, txnCtx, function), "AUnorderedList");
+            }
+        }
+        return null;
+    }
+
+    private static IResultTypeComputer getResultTypeComputer(final MetadataTransactionContext txnCtx,
+            final Function function) throws MetadataException {
+
+        final IAType type = getTypeInfo(function.getReturnType(), txnCtx, function);
+        switch (type.getTypeTag()) {
+            case INT32:
+                return AInt32TypeComputer.INSTANCE;
+            case FLOAT:
+                return AFloatTypeComputer.INSTANCE;
+            case DOUBLE:
+                return ADoubleTypeComputer.INSTANCE;
+            case STRING:
+                return AStringTypeComputer.INSTANCE;
+            case ORDEREDLIST:
+                return new IResultTypeComputer() {
+                    @Override
+                    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+                            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+
+                        return new AOrderedListType(((AOrderedListType) type).getItemType(), ((AOrderedListType) type)
+                                .getItemType().getTypeName());
+                    }
+
+                };
+            case UNORDEREDLIST:
+                return new IResultTypeComputer() {
+                    @Override
+                    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+                            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+
+                        return new AUnorderedListType(type, type.getTypeName());
+                    }
+
+                };
+            default:
+                IResultTypeComputer typeComputer = new IResultTypeComputer() {
+                    @Override
+                    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+                            IMetadataProvider<?, ?> mp) throws AlgebricksException {
+                        return type;
+                    }
+                };
+                return typeComputer;
+        }
+
+    }
+
+    private static IAType getType(Function function, MetadataTransactionContext txnCtx) throws AlgebricksException {
+        IAType collectionType = null;
+        try {
+            collectionType = getCollectionType(function.getReturnType(), txnCtx, function);
+            if (collectionType != null) {
+                return collectionType;
+            } else {
+
+                Datatype datatype;
+                datatype = MetadataManager.INSTANCE.getDatatype(txnCtx, function.getDataverseName(),
+                        function.getReturnType());
+                return datatype.getDatatype();
+            }
+        } catch (MetadataException me) {
+            throw new AlgebricksException(me);
+        }
+    }
+
+    private static IFunctionInfo getUnnestFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    private static IFunctionInfo getStatefulFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    private static IFunctionInfo getAggregateFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+    
+    public static void main(String args[]) throws FileNotFoundException, IOException {
+        ExternalFunctionCompilerUtil obj = new ExternalFunctionCompilerUtil();
+        ObjectOutputStream oos = new ObjectOutputStream(new FileOutputStream("/tmp/ecu.obj"));
+        oos.writeObject(obj);
+    }
+
+}
\ No newline at end of file
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 d5d4cc2..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
@@ -40,6 +40,8 @@
         AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.DATASET);
         AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INGEST, false);
         AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INGEST);
+        AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INTERCEPT, false);
+        AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INTERCEPT);
     }
 
     public static void addMetadataBuiltinFunctions() {
@@ -93,6 +95,44 @@
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
                     IMetadataProvider<?, ?> mp) throws AlgebricksException {
                 AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
+                if (f.getArguments().size() != 3) {
+                    throw new AlgebricksException("Incorrect number of arguments -> arity is 3, not "
+                            + f.getArguments().size());
+                }
+                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 feed-ingest argument.");
+                }
+                if (a1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                    return BuiltinType.ANY;
+                }
+                AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
+                String typeArg = ((AString) acv.getObject()).getStringValue();
+                AqlMetadataProvider metadata = ((AqlMetadataProvider) mp);
+                Pair<String, String> argInfo = getDatasetInfo(metadata, typeArg);
+                String dataverseName = argInfo.first;
+                String typeName = argInfo.second;
+                if (dataverseName == null) {
+                    throw new AlgebricksException("Unspecified dataverse!");
+                }
+                IAType t2 = metadata.findType(dataverseName, typeName);
+                if (t2 == null) {
+                    throw new AlgebricksException("Unknown type  " + typeName);
+                }
+                return t2;
+            }
+        });
+
+        AsterixBuiltinFunctions.addFunction(AsterixBuiltinFunctions.FEED_INTERCEPT, new IResultTypeComputer() {
+
+            @Override
+            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());
                 }
@@ -132,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/dataflow/data/common/AqlExpressionTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
index 7eb2f30..12b2753 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
@@ -17,8 +17,10 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.AsterixExternalFunctionInfo;
 import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
 import edu.uci.ics.asterix.om.types.AUnionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -72,8 +74,14 @@
             unionList.add(BuiltinType.ABOOLEAN);
             return new AUnionType(unionList, "OptionalBoolean");
         }
-        // Note: only builtin functions, for now.
-        IResultTypeComputer rtc = AsterixBuiltinFunctions.getResultTypeComputer(fi);
+        // Note: built-in functions + udfs
+        IResultTypeComputer rtc = null;
+        FunctionSignature signature = new FunctionSignature(fi.getNamespace(), fi.getName(), fi.getArity());
+        if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(signature, true)) {
+            rtc = AsterixBuiltinFunctions.getResultTypeComputer(fi);
+        } else {
+            rtc = ((AsterixExternalFunctionInfo) expr.getFunctionInfo()).getResultTypeComputer();
+        }
         if (rtc == null) {
             throw new AlgebricksException("Type computer missing for " + fi);
         }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
index fe0086b..7cb490c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
@@ -20,15 +20,20 @@
 
 public final class AMutableUnorderedList extends AUnorderedList {
 
-    public AMutableUnorderedList(AUnorderedListType type) {
-        super(type);
-    }
+	public AMutableUnorderedList(AUnorderedListType type) {
+		super(type);
+	}
 
-    public AMutableUnorderedList(AUnorderedListType type, ArrayList<IAObject> sequence) {
-        super(type, sequence);
-    }
+	public AMutableUnorderedList(AUnorderedListType type,
+			ArrayList<IAObject> sequence) {
+		super(type, sequence);
+	}
 
-    public void add(IAObject obj) {
-        values.add(obj);
-    }
+	public void add(IAObject obj) {
+		values.add(obj);
+	}
+
+	public void clear() {
+		values.clear();
+	}
 }
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 3c8a0c4..540530e 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
@@ -49,6 +49,7 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
+import edu.uci.ics.asterix.om.typecomputer.impl.FlowRecordResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedCollectionMemberResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedFieldAccessByNameResultType;
@@ -239,7 +240,9 @@
 
     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);
 
     public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "index-search", FunctionIdentifier.VARARGS);
@@ -494,6 +497,8 @@
             "inject-failure", 2);
     public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "cast-record", 1);
+    public final static FunctionIdentifier FLOW_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "flow-record", 1);
     public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "cast-list", 1);
 
@@ -811,7 +816,6 @@
         addPrivateFunction(TYPE_OF, null);
         addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorResultType.INSTANCE);
         addFunction(WORD_TOKENS, new IResultTypeComputer() {
-
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
                     IMetadataProvider<?, ?> mp) throws AlgebricksException {
@@ -960,12 +964,14 @@
     static {
         datasetFunctions.add(getAsterixFunctionInfo(DATASET));
         datasetFunctions.add(getAsterixFunctionInfo(FEED_INGEST));
+        datasetFunctions.add(getAsterixFunctionInfo(FEED_INTERCEPT));
         datasetFunctions.add(getAsterixFunctionInfo(INDEX_SEARCH));
     }
 
     static {
         addUnnestFun(DATASET, false);
         addUnnestFun(FEED_INGEST, false);
+        addUnnestFun(FEED_INTERCEPT, false);
         addUnnestFun(RANGE, true);
         addUnnestFun(SCAN_COLLECTION, false);
         addUnnestFun(SUBSET_COLLECTION, false);
@@ -979,9 +985,10 @@
         return datasetFunctions.contains(getAsterixFunctionInfo(fi));
     }
 
+    /*
     public static boolean isBuiltinCompilerFunction(FunctionIdentifier fi, boolean includePrivateFunctions) {
         return builtinPublicFunctionsSet.keySet().contains(getAsterixFunctionInfo(fi));
-    }
+    }*/
 
     public static boolean isBuiltinCompilerFunction(FunctionSignature signature, boolean includePrivateFunctions) {
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
index 186ca78..c536a88 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
@@ -14,11 +14,13 @@
  */
 package edu.uci.ics.asterix.om.functions;
 
+import java.io.Serializable;
+
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 
-public class AsterixFunctionInfo implements IFunctionInfo {
+public class AsterixFunctionInfo implements IFunctionInfo, Serializable {
 
     private final FunctionIdentifier functionIdentifier;
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
index bd75f9c..3bc7792 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.asterix.om.typecomputer.base;
 
+import java.io.Serializable;
 
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -21,7 +22,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public interface IResultTypeComputer {
+public interface IResultTypeComputer extends Serializable {
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException;
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java
new file mode 100644
index 0000000..a982a1f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+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.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class FlowRecordResultTypeComputer implements IResultTypeComputer {
+
+    public static final FlowRecordResultTypeComputer INSTANCE = new FlowRecordResultTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expression;
+        IAType type = TypeComputerUtilities.getRequiredType(funcExpr);
+        if (type == null) {
+            type = (IAType) env.getType(funcExpr.getArguments().get(0).getValue());
+        }
+        return type;
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 0f836af..b8b3226 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -46,8 +47,9 @@
     private AsterixMetadataProperties metadataProperties;
     private AsterixStorageProperties storageProperties;
     private AsterixTransactionProperties txnProperties;
+    private IHyracksClientConnection hcc;
 
-    public static void initialize(ICCApplicationContext ccAppCtx) throws AsterixException {
+    public static void initialize(ICCApplicationContext ccAppCtx, IHyracksClientConnection hcc) throws AsterixException {
         if (INSTANCE == null) {
             INSTANCE = new AsterixAppContextInfo(ccAppCtx);
         }
@@ -57,6 +59,7 @@
         INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
         INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
         INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+        INSTANCE.hcc = hcc;
         Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
     }
 
@@ -98,6 +101,10 @@
         return externalProperties;
     }
 
+    public IHyracksClientConnection getHcc() {
+        return hcc;
+    }
+
     @Override
     public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
         return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
@@ -107,4 +114,5 @@
     public IStorageManagerInterface getStorageManagerInterface() {
         return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
+
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
index 6e6e09a..9af206bb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
@@ -14,11 +14,25 @@
  */
 package edu.uci.ics.asterix.om.util;
 
+import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+
 /**
  * A holder class for properties related to the Asterix cluster.
  */
@@ -27,13 +41,31 @@
 
     private static final Logger LOGGER = Logger.getLogger(AsterixClusterProperties.class.getName());
 
-    private static final String IO_DEVICES = "iodevices";
-
     public static final AsterixClusterProperties INSTANCE = new AsterixClusterProperties();
+    public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
+
+    private static final String IO_DEVICES = "iodevices";
 
     private Map<String, Map<String, String>> ncConfiguration = new HashMap<String, Map<String, String>>();
 
+    private final Cluster cluster;
+
+    private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
+
     private AsterixClusterProperties() {
+        InputStream is = this.getClass().getClassLoader().getResourceAsStream(CLUSTER_CONFIGURATION_FILE);
+        if (is != null) {
+            try {
+                JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+                Unmarshaller unmarshaller = ctx.createUnmarshaller();
+                cluster = (Cluster) unmarshaller.unmarshal(is);
+
+            } catch (JAXBException e) {
+                throw new IllegalStateException("Failed to read configuration file " + CLUSTER_CONFIGURATION_FILE);
+            }
+        } else {
+            cluster = null;
+        }
     }
 
     public enum State {
@@ -43,12 +75,13 @@
 
     private State state = State.UNUSABLE;
 
-    public void removeNCConfiguration(String nodeId) {
-        state = State.UNUSABLE;
+    public synchronized void removeNCConfiguration(String nodeId) {
+        // state = State.UNUSABLE;
         ncConfiguration.remove(nodeId);
+        resetClusterPartitionConstraint();
     }
 
-    public void addNCConfiguration(String nodeId, Map<String, String> configuration) {
+    public synchronized void addNCConfiguration(String nodeId, Map<String, String> configuration) {
         ncConfiguration.put(nodeId, configuration);
         if (ncConfiguration.keySet().size() == AsterixAppContextInfo.getInstance().getMetadataProperties()
                 .getNodeNames().size()) {
@@ -57,6 +90,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(" Registering configuration parameters for node id" + nodeId);
         }
+        resetClusterPartitionConstraint();
     }
 
     /**
@@ -64,10 +98,11 @@
      * 
      * @param nodeId
      *            unique identifier of the Node Controller
-     * @return number of IO devices. -1 if the node id is not valid. A node id is not valid
-     *         if it does not correspond to the set of registered Node Controllers.
+     * @return number of IO devices. -1 if the node id is not valid. A node id
+     *         is not valid if it does not correspond to the set of registered
+     *         Node Controllers.
      */
-    public int getNumberOfIODevices(String nodeId) {
+    public synchronized int getNumberOfIODevices(String nodeId) {
         Map<String, String> ncConfig = ncConfiguration.get(nodeId);
         if (ncConfig == null) {
             if (LOGGER.isLoggable(Level.WARNING)) {
@@ -78,7 +113,7 @@
         }
         return ncConfig.get(IO_DEVICES).split(",").length;
     }
-    
+
     /**
      * Returns the IO devices configured for a Node Controller
      * 
@@ -103,4 +138,105 @@
         return state;
     }
 
+    public Cluster getCluster() {
+        return cluster;
+    }
+
+    public synchronized Node getAvailableSubstitutionNode() {
+        List<Node> subNodes = cluster.getSubstituteNodes() == null ? null : cluster.getSubstituteNodes().getNode();
+        return subNodes == null || subNodes.isEmpty() ? null : subNodes.get(0);
+    }
+
+    public synchronized Set<String> getParticipantNodes() {
+        Set<String> participantNodes = new HashSet<String>();
+        for (String pNode : ncConfiguration.keySet()) {
+            participantNodes.add(pNode);
+        }
+        return participantNodes;
+    }
+
+    public synchronized AlgebricksPartitionConstraint getClusterLocations() {
+        if (clusterPartitionConstraint == null) {
+            resetClusterPartitionConstraint();
+        }
+        return clusterPartitionConstraint;
+    }
+
+    private synchronized void resetClusterPartitionConstraint() {
+        Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+        ArrayList<String> locs = new ArrayList<String>();
+        for (String i : stores.keySet()) {
+            String[] nodeStores = stores.get(i);
+            int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+            for (int j = 0; j < nodeStores.length; j++) {
+                for (int k = 0; k < numIODevices; k++) {
+                    locs.add(i);
+                }
+            }
+        }
+        String[] cluster = new String[locs.size()];
+        cluster = locs.toArray(cluster);
+        clusterPartitionConstraint = new AlgebricksAbsolutePartitionConstraint(cluster);
+    }
+
+    private static class AsterixCluster {
+
+        private final String asterixInstance;
+        private Map<String, AsterixNode> asterixNodes;
+
+        public AsterixCluster(Cluster cluster) {
+            asterixInstance = cluster.getInstanceName();
+            asterixNodes = new HashMap<String, AsterixNode>();
+            for (Node node : cluster.getNode()) {
+                AsterixNode aNode = new AsterixNode(node, AsterixNode.NodeRole.PARTICIPANT,
+                        AsterixNode.NodeState.INACTIVE);
+                asterixNodes.put(asterixInstance + "_" + node.getId(), aNode);
+            }
+
+            for (Node node : cluster.getSubstituteNodes().getNode()) {
+                AsterixNode aNode = new AsterixNode(node, AsterixNode.NodeRole.SUBSTITUTE,
+                        AsterixNode.NodeState.INACTIVE);
+                asterixNodes.put(asterixInstance + "_" + node.getId(), aNode);
+            }
+        }
+
+        private static class AsterixNode {
+
+            private final Node node;
+            private NodeRole role;
+            private NodeState state;
+
+            public enum NodeRole {
+                PARTICIPANT,
+                SUBSTITUTE
+            }
+
+            public enum NodeState {
+                ACTIVE,
+                INACTIVE
+            }
+
+            public AsterixNode(Node node, NodeRole role, NodeState state) {
+                this.node = node;
+                this.role = role;
+                this.state = state;
+            }
+
+            @Override
+            public String toString() {
+                return node.getId() + "_" + role + "_" + state;
+            }
+        }
+
+        public void notifyChangeState(String nodeId, AsterixNode.NodeRole newRole, AsterixNode.NodeState newState) {
+            AsterixNode node = asterixNodes.get(nodeId);
+            if (node != null) {
+                node.role = newRole;
+                node.state = newState;
+            } else {
+                throw new IllegalStateException("Unknown nodeId" + nodeId);
+            }
+
+        }
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
new file mode 100644
index 0000000..81c23cd
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
@@ -0,0 +1,74 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class FlowRecordDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new FlowRecordDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType inputType;
+
+    public void reset(ARecordType inputType) {
+        this.inputType = inputType;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.FLOW_RECORD;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        final ICopyEvaluatorFactory recordEvalFactory = args[0];
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                final DataOutput out = output.getDataOutput();
+                final ArrayBackedValueStorage recordBuffer = new ArrayBackedValueStorage();
+                final ICopyEvaluator recEvaluator = recordEvalFactory.createEvaluator(recordBuffer);
+
+                return new ICopyEvaluator() {
+                    // pointable allocator
+                    private PointableAllocator allocator = new PointableAllocator();
+                    final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            recordBuffer.reset();
+                            recEvaluator.evaluate(tuple);
+                            recAccessor.set(recordBuffer);
+                            out.write(recAccessor.getByteArray(), recAccessor.getStartOffset(), recAccessor.getLength());
+                        } catch (Exception ioe) {
+                            throw new AlgebricksException(ioe);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 566aa98..011836a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -151,6 +151,7 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.EndsWithDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.FieldAccessByIndexDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.FieldAccessByNameDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.FlowRecordDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.FuzzyEqDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.GetItemDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.GramTokensDescriptor;
@@ -494,6 +495,7 @@
         temp.add(InjectFailureDescriptor.FACTORY);
         temp.add(CastListDescriptor.FACTORY);
         temp.add(CastRecordDescriptor.FACTORY);
+        temp.add(FlowRecordDescriptor.FACTORY);
         temp.add(NotNullDescriptor.FACTORY);
 
         // Spatial and temporal type accessors
@@ -741,6 +743,10 @@
             }
             ((CastListDescriptor) fd).reset(rt, (AbstractCollectionType) it);
         }
+        if (fd.getIdentifier().equals(AsterixBuiltinFunctions.FLOW_RECORD)) {
+            ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
+            ((FlowRecordDescriptor) fd).reset(it);
+        }
         if (fd.getIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
             ARecordType rt = (ARecordType) context.getType(expr);
             ((OpenRecordConstructorDescriptor) fd).reset(rt,
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 2322338..ab52939 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -18,6 +18,8 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -36,6 +38,8 @@
  */
 public abstract class AbstractTupleParser implements ITupleParser {
 
+    protected static Logger LOGGER = Logger.getLogger(AbstractTupleParser.class.getName());
+
     protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
     protected DataOutput dos = tb.getDataOutput();
     protected final FrameTupleAppender appender;
@@ -68,6 +72,7 @@
                 addTupleToFrame(writer);
             }
             if (appender.getTupleCount() > 0) {
+
                 FrameUtils.flushFrame(frame, writer);
             }
         } catch (AsterixException ae) {
diff --git a/asterix-test-framework/pom.xml b/asterix-test-framework/pom.xml
old mode 100755
new mode 100644
diff --git a/asterix-test-framework/src/main/resources/Catalog.xsd b/asterix-test-framework/src/main/resources/Catalog.xsd
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.all.first.cleaned b/asterix-tools/data/uscensus/dist.all.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.all.last.cleaned b/asterix-tools/data/uscensus/dist.all.last.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.female.first.cleaned b/asterix-tools/data/uscensus/dist.female.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.male.first.cleaned b/asterix-tools/data/uscensus/dist.male.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/pom.xml b/asterix-tools/pom.xml
index 6c7db41..861dd79 100644
--- a/asterix-tools/pom.xml
+++ b/asterix-tools/pom.xml
@@ -148,6 +148,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-external-data</artifactId>
+			<version>0.8.1-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.httpcomponents</groupId>
 			<artifactId>httpclient</artifactId>
 			<version>4.2.2</version>
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java
new file mode 100644
index 0000000..0c93564
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java
@@ -0,0 +1,2476 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class DataGenerator {
+
+    private static RandomDateGenerator randDateGen;
+    private static RandomNameGenerator randNameGen;
+    private static RandomEmploymentGenerator randEmpGen;
+    private static RandomMessageGenerator randMessageGen;
+    private static RandomLocationGenerator randLocationGen;
+
+    private static DistributionHandler fbDistHandler;
+    private static DistributionHandler twDistHandler;
+
+    private static int totalFbMessages;
+    private static int numFbOnlyUsers;
+    private static int totalTwMessages;
+    private static int numTwOnlyUsers;
+
+    private static int numCommonUsers;
+
+    private static int fbUserId;
+    private static int twUserId;
+
+    private static int fbMessageId;
+    private static int twMessageId;
+
+    private static Random random = new Random();
+
+    private static String commonUserFbSuffix = "_fb";
+    private static String commonUserTwSuffix = "_tw";
+
+    private static String outputDir;
+
+    private static PartitionConfiguration partition;
+
+    private static FacebookUser fbUser = new FacebookUser();
+    private static TwitterUser twUser = new TwitterUser();
+
+    private static FacebookMessage fbMessage = new FacebookMessage();
+    private static TweetMessage twMessage = new TweetMessage();
+
+    private static int duration;
+
+    private static DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+
+    private static void generateFacebookOnlyUsers(int numFacebookUsers) throws IOException {
+        FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, true);
+        FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, true);
+
+        for (int i = 0; i < numFacebookUsers; i++) {
+            getFacebookUser(null);
+            appender.appendToFile(fbUser.toString());
+            generateFacebookMessages(fbUser, messageAppender, -1);
+        }
+        appender.close();
+        messageAppender.close();
+    }
+
+    private static void generateTwitterOnlyUsers(int numTwitterUsers) throws IOException {
+        FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, true);
+        FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, true);
+
+        for (int i = 0; i < numTwitterUsers; i++) {
+            getTwitterUser(null);
+            appender.appendToFile(twUser.toString());
+            generateTwitterMessages(twUser, messageAppender, -1);
+        }
+        appender.close();
+        messageAppender.close();
+    }
+
+    private static void generateCommonUsers() throws IOException {
+        FileAppender fbAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, false);
+        FileAppender twAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, false);
+        FileAppender fbMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, false);
+        FileAppender twMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, false);
+
+        for (int i = 0; i < numCommonUsers; i++) {
+            getFacebookUser(commonUserFbSuffix);
+            fbAppender.appendToFile(fbUser.toString());
+            generateFacebookMessages(fbUser, fbMessageAppender, -1);
+
+            getCorrespondingTwitterUser(fbUser);
+            twAppender.appendToFile(twUser.toString());
+            generateTwitterMessages(twUser, twMessageAppender, -1);
+        }
+
+        fbAppender.close();
+        twAppender.close();
+        fbMessageAppender.close();
+        twMessageAppender.close();
+    }
+
+    private static void generateFacebookMessages(FacebookUser user, FileAppender appender, int numMsg)
+            throws IOException {
+        Message message;
+        int numMessages = 0;
+        if (numMsg == -1) {
+            numMessages = fbDistHandler
+                    .getFromDistribution(fbUserId - partition.getTargetPartition().getFbUserKeyMin());
+        }
+        for (int i = 0; i < numMessages; i++) {
+            message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            fbMessage.reset(fbMessageId++, user.getId(), random.nextInt(totalFbMessages + 1), location, message);
+            appender.appendToFile(fbMessage.toString());
+        }
+    }
+
+    private static void generateTwitterMessages(TwitterUser user, FileAppender appender, int numMsg) throws IOException {
+        Message message;
+        int numMessages = 0;
+        if (numMsg == -1) {
+            numMessages = twDistHandler
+                    .getFromDistribution(twUserId - partition.getTargetPartition().getTwUserKeyMin());
+        }
+
+        for (int i = 0; i < numMessages; i++) {
+            message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            DateTime sendTime = randDateGen.getNextRandomDatetime();
+            twMessage.reset(twMessageId + "", user, location, sendTime, message.getReferredTopics(), message);
+            twMessageId++;
+            appender.appendToFile(twMessage.toString());
+        }
+    }
+
+    public static Iterator<TweetMessage> getTwitterMessageIterator() {
+        return new TweetMessageIterator(duration);
+    }
+
+    public static class TweetMessageIterator implements Iterator<TweetMessage> {
+
+        private final int duration;
+        private long startTime = 0;
+
+        public TweetMessageIterator(int duration) {
+            this.duration = duration;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (startTime == 0) {
+                startTime = System.currentTimeMillis();
+            }
+            return System.currentTimeMillis() - startTime < duration * 1000;
+        }
+
+        @Override
+        public TweetMessage next() {
+            getTwitterUser(null);
+            Message message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            DateTime sendTime = randDateGen.getNextRandomDatetime();
+            twMessage.reset(UUID.randomUUID().toString(), twUser, location, sendTime, message.getReferredTopics(),
+                    message);
+            twMessageId++;
+            if (twUserId > numTwOnlyUsers) {
+                twUserId = 1;
+            }
+            return twMessage;
+
+        }
+
+        @Override
+        public void remove() {
+            // TODO Auto-generated method stub
+
+        }
+
+    }
+
+    public static class InitializationInfo {
+        public Date startDate = new Date(1, 1, 2005);
+        public Date endDate = new Date(8, 20, 2012);
+        public String[] lastNames = DataGenerator.lastNames;
+        public String[] firstNames = DataGenerator.firstNames;
+        public String[] vendors = DataGenerator.vendors;
+        public String[] jargon = DataGenerator.jargon;
+        public String[] org_list = DataGenerator.org_list;
+        public int percentEmployed = 90;
+        public Date employmentStartDate = new Date(1, 1, 2000);
+        public Date employmentEndDate = new Date(31, 12, 2012);
+        public int totalFbMessages;
+        public int numFbOnlyUsers;
+        public int totalTwMessages;
+        public int numTwOnlyUsers = 5000;
+        public int numCommonUsers;
+        public int fbUserIdMin;
+        public int fbMessageIdMin;
+        public int twUserIdMin;
+        public int twMessageIdMin;
+        public int timeDurationInSecs = 60;
+
+    }
+
+    public static void initialize(InitializationInfo info) {
+        randDateGen = new RandomDateGenerator(info.startDate, info.endDate);
+        randNameGen = new RandomNameGenerator(info.firstNames, info.lastNames);
+        randEmpGen = new RandomEmploymentGenerator(info.percentEmployed, info.employmentStartDate,
+                info.employmentEndDate, info.org_list);
+        randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+        randMessageGen = new RandomMessageGenerator(info.vendors, info.jargon);
+        fbDistHandler = new DistributionHandler(info.totalFbMessages, 0.5, info.numFbOnlyUsers + info.numCommonUsers);
+        twDistHandler = new DistributionHandler(info.totalTwMessages, 0.5, info.numTwOnlyUsers + info.numCommonUsers);
+        fbUserId = info.fbUserIdMin;
+        twUserId = info.twUserIdMin;
+
+        fbMessageId = info.fbMessageIdMin;
+        twMessageId = info.fbMessageIdMin;
+        duration = info.timeDurationInSecs;
+    }
+
+    public static void main(String args[]) throws Exception {
+
+        String controllerInstallDir = null;
+        if (args.length < 2) {
+            printUsage();
+            System.exit(1);
+        } else {
+            controllerInstallDir = args[0];
+            String partitionConfXML = controllerInstallDir + "/output/partition-conf.xml";
+            String partitionName = args[1];
+            partition = XMLUtil.getPartitionConfiguration(partitionConfXML, partitionName);
+        }
+
+        // 1
+        randDateGen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+
+        String firstNameFile = controllerInstallDir + "/metadata/firstNames.txt";
+        String lastNameFile = controllerInstallDir + "/metadata/lastNames.txt";
+        String vendorFile = controllerInstallDir + "/metadata/vendors.txt";
+        String jargonFile = controllerInstallDir + "/metadata/jargon.txt";
+        String orgList = controllerInstallDir + "/metadata/org_list.txt";
+
+        randNameGen = new RandomNameGenerator(firstNameFile, lastNameFile);
+        randEmpGen = new RandomEmploymentGenerator(90, new Date(1, 1, 2000), new Date(8, 20, 2012), orgList);
+        randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+        randMessageGen = new RandomMessageGenerator(vendorFile, jargonFile);
+
+        totalFbMessages = partition.getTargetPartition().getFbMessageIdMax()
+                - partition.getTargetPartition().getFbMessageIdMin() + 1;
+        numFbOnlyUsers = (partition.getTargetPartition().getFbUserKeyMax()
+                - partition.getTargetPartition().getFbUserKeyMin() + 1)
+                - partition.getTargetPartition().getCommonUsers();
+
+        totalTwMessages = partition.getTargetPartition().getTwMessageIdMax()
+                - partition.getTargetPartition().getTwMessageIdMin() + 1;
+        numTwOnlyUsers = (partition.getTargetPartition().getTwUserKeyMax()
+                - partition.getTargetPartition().getTwUserKeyMin() + 1)
+                - partition.getTargetPartition().getCommonUsers();
+
+        numCommonUsers = partition.getTargetPartition().getCommonUsers();
+        fbDistHandler = new DistributionHandler(totalFbMessages, 0.5, numFbOnlyUsers + numCommonUsers);
+        twDistHandler = new DistributionHandler(totalTwMessages, 0.5, numTwOnlyUsers + numCommonUsers);
+
+        fbUserId = partition.getTargetPartition().getFbUserKeyMin();
+        twUserId = partition.getTargetPartition().getTwUserKeyMin();
+
+        fbMessageId = partition.getTargetPartition().getFbMessageIdMin();
+        twMessageId = partition.getTargetPartition().getTwMessageIdMin();
+
+        outputDir = partition.getSourcePartition().getPath();
+        generateData();
+    }
+
+    public static void printUsage() {
+        System.out.println(" Error: Invalid number of arguments ");
+        System.out.println(" Usage :" + " DataGenerator <path to configuration file> <partition name> ");
+    }
+
+    public static void generateData() throws IOException {
+        generateFacebookOnlyUsers(numFbOnlyUsers);
+        generateTwitterOnlyUsers(numTwOnlyUsers);
+        generateCommonUsers();
+        System.out.println("Partition :" + partition.getTargetPartition().getName() + " finished");
+    }
+
+    public static void getFacebookUser(String usernameSuffix) {
+        String suggestedName = randNameGen.getRandomName();
+        String[] nameComponents = suggestedName.split(" ");
+        String name = nameComponents[0] + nameComponents[1];
+        if (usernameSuffix != null) {
+            name = name + usernameSuffix;
+        }
+        String alias = nameComponents[0];
+        String userSince = randDateGen.getNextRandomDatetime().toString();
+        int numFriends = random.nextInt(25);
+        int[] friendIds = RandomUtil.getKFromN(numFriends, (numFbOnlyUsers + numCommonUsers));
+        Employment emp = randEmpGen.getRandomEmployment();
+        fbUser.reset(fbUserId++, alias, name, userSince, friendIds, emp);
+    }
+
+    public static void getTwitterUser(String usernameSuffix) {
+        String suggestedName = randNameGen.getRandomName();
+        String[] nameComponents = suggestedName.split(" ");
+        String screenName = nameComponents[0] + nameComponents[1] + randNameGen.getRandomNameSuffix();
+        String name = suggestedName;
+        if (usernameSuffix != null) {
+            name = name + usernameSuffix;
+        }
+        int numFriends = random.nextInt((int) (100)); // draw from Zipfian
+        int statusesCount = random.nextInt(500); // draw from Zipfian
+        int followersCount = random.nextInt((int) (200));
+        twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+        twUserId++;
+    }
+
+    public static void getCorrespondingTwitterUser(FacebookUser fbUser) {
+        String screenName = fbUser.getName().substring(0, fbUser.getName().lastIndexOf(commonUserFbSuffix))
+                + commonUserTwSuffix;
+        String name = screenName.split(" ")[0];
+        int numFriends = random.nextInt((int) ((numTwOnlyUsers + numCommonUsers)));
+        int statusesCount = random.nextInt(500); // draw from Zipfian
+        int followersCount = random.nextInt((int) (numTwOnlyUsers + numCommonUsers));
+        twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+    }
+
+    public static class RandomDateGenerator {
+
+        private final Date startDate;
+        private final Date endDate;
+        private final Random random = new Random();
+        private final int yearDifference;
+        private Date workingDate;
+        private Date recentDate;
+        private DateTime dateTime;
+
+        public RandomDateGenerator(Date startDate, Date endDate) {
+            this.startDate = startDate;
+            this.endDate = endDate;
+            yearDifference = endDate.getYear() - startDate.getYear() + 1;
+            workingDate = new Date();
+            recentDate = new Date();
+            dateTime = new DateTime();
+        }
+
+        public Date getStartDate() {
+            return startDate;
+        }
+
+        public Date getEndDate() {
+            return endDate;
+        }
+
+        public Date getNextRandomDate() {
+            int year = random.nextInt(yearDifference) + startDate.getYear();
+            int month;
+            int day;
+            if (year == endDate.getYear()) {
+                month = random.nextInt(endDate.getMonth()) + 1;
+                if (month == endDate.getMonth()) {
+                    day = random.nextInt(endDate.getDay()) + 1;
+                } else {
+                    day = random.nextInt(28) + 1;
+                }
+            } else {
+                month = random.nextInt(12) + 1;
+                day = random.nextInt(28) + 1;
+            }
+            workingDate.reset(month, day, year);
+            return workingDate;
+        }
+
+        public DateTime getNextRandomDatetime() {
+            Date randomDate = getNextRandomDate();
+            dateTime.reset(randomDate);
+            return dateTime;
+        }
+
+        public Date getNextRecentDate(Date date) {
+            int year = date.getYear()
+                    + (date.getYear() == endDate.getYear() ? 0 : random.nextInt(endDate.getYear() - date.getYear()));
+            int month = (year == endDate.getYear()) ? date.getMonth() == endDate.getMonth() ? (endDate.getMonth())
+                    : (date.getMonth() + random.nextInt(endDate.getMonth() - date.getMonth())) : random.nextInt(12) + 1;
+
+            int day = (year == endDate.getYear()) ? month == endDate.getMonth() ? date.getDay() == endDate.getDay() ? endDate
+                    .getDay() : date.getDay() + random.nextInt(endDate.getDay() - date.getDay())
+                    : random.nextInt(28) + 1
+                    : random.nextInt(28) + 1;
+            recentDate.reset(month, day, year);
+            return recentDate;
+        }
+
+        public static void main(String args[]) throws Exception {
+            Date date = new Date(2, 20, 2012);
+            RandomDateGenerator dgen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+            while (true) {
+                Date nextDate = dgen.getNextRandomDate();
+                if (nextDate.getDay() == 0) {
+                    throw new Exception("invalid date " + nextDate);
+                }
+
+                // System.out.println(" original date: " + date);
+                System.out.println(nextDate);
+            }
+        }
+    }
+
+    public static class DateTime extends Date {
+
+        private String hour = "10";
+        private String min = "10";
+        private String sec = "00";
+        private long chrononTime;
+
+        public DateTime(int month, int day, int year, String hour, String min, String sec) {
+            super(month, day, year);
+            this.hour = hour;
+            this.min = min;
+            this.sec = sec;
+            chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+                    Integer.parseInt(sec)).getTime();
+        }
+
+        public void reset(int month, int day, int year, String hour, String min, String sec) {
+            super.setDay(month);
+            super.setDay(day);
+            super.setYear(year);
+            this.hour = hour;
+            this.min = min;
+            this.sec = sec;
+            chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+                    Integer.parseInt(sec)).getTime();
+        }
+
+        public DateTime() {
+        }
+
+        public DateTime(Date date) {
+            super(date.getMonth(), date.getDay(), date.getYear());
+        }
+
+        public void reset(Date date) {
+            reset(date.getMonth(), date.getDay(), date.getYear());
+        }
+
+        public DateTime(Date date, int hour, int min, int sec) {
+            super(date.getMonth(), date.getDay(), date.getYear());
+            this.hour = (hour < 10) ? "0" : "" + hour;
+            this.min = (min < 10) ? "0" : "" + min;
+            this.sec = (sec < 10) ? "0" : "" + sec;
+        }
+
+        public long getChrononTime() {
+            return chrononTime;
+        }
+
+        public String getHour() {
+            return hour;
+        }
+
+        public String getMin() {
+            return min;
+        }
+
+        public String getSec() {
+            return sec;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("datetime");
+            builder.append("(\"");
+            builder.append(super.getYear());
+            builder.append("-");
+            builder.append(super.getMonth() < 10 ? "0" + super.getMonth() : super.getMonth());
+            builder.append("-");
+            builder.append(super.getDay() < 10 ? "0" + super.getDay() : super.getDay());
+            builder.append("T");
+            builder.append(hour + ":" + min + ":" + sec);
+            builder.append("\")");
+            return builder.toString();
+        }
+    }
+
+    public static class Message {
+
+        private char[] message = new char[500];
+        private List<String> referredTopics;
+        private int length;
+
+        public Message(char[] m, List<String> referredTopics) {
+            System.arraycopy(m, 0, message, 0, m.length);
+            length = m.length;
+            this.referredTopics = referredTopics;
+        }
+
+        public Message() {
+            referredTopics = new ArrayList<String>();
+            length = 0;
+        }
+
+        public char[] getMessage() {
+            return message;
+        }
+
+        public List<String> getReferredTopics() {
+            return referredTopics;
+        }
+
+        public void reset(char[] m, int offset, int length, List<String> referredTopics) {
+            System.arraycopy(m, offset, message, 0, length);
+            this.length = length;
+            this.referredTopics = referredTopics;
+        }
+
+        public int getLength() {
+            return length;
+        }
+
+        public char charAt(int index) {
+            return message[index];
+        }
+
+    }
+
+    public static class Point {
+
+        private float latitude;
+        private float longitude;
+
+        public float getLatitude() {
+            return latitude;
+        }
+
+        public float getLongitude() {
+            return longitude;
+        }
+
+        public Point(float latitude, float longitude) {
+            this.latitude = latitude;
+            this.longitude = longitude;
+        }
+
+        public void reset(float latitude, float longitude) {
+            this.latitude = latitude;
+            this.longitude = longitude;
+        }
+
+        public Point() {
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("point(\"" + latitude + "," + longitude + "\")");
+            return builder.toString();
+        }
+    }
+
+    public static class RandomNameGenerator {
+
+        private String[] firstNames;
+        private String[] lastNames;
+
+        private final Random random = new Random();
+
+        private final String[] connectors = new String[] { "_", "#", "$", "@" };
+
+        public RandomNameGenerator(String firstNameFilePath, String lastNameFilePath) throws IOException {
+            firstNames = FileUtil.listyFile(new File(firstNameFilePath)).toArray(new String[] {});
+            lastNames = FileUtil.listyFile(new File(lastNameFilePath)).toArray(new String[] {});
+        }
+
+        public RandomNameGenerator(String[] firstNames, String[] lastNames) {
+            this.firstNames = firstNames;
+            this.lastNames = lastNames;
+        }
+
+        public String getRandomName() {
+            String name;
+            name = getSuggestedName();
+            return name;
+
+        }
+
+        private String getSuggestedName() {
+            int firstNameIndex = random.nextInt(firstNames.length);
+            int lastNameIndex = random.nextInt(lastNames.length);
+            String suggestedName = firstNames[firstNameIndex] + " " + lastNames[lastNameIndex];
+            return suggestedName;
+        }
+
+        public String getRandomNameSuffix() {
+            return connectors[random.nextInt(connectors.length)] + random.nextInt(1000);
+        }
+    }
+
+    public static class RandomMessageGenerator {
+
+        private final MessageTemplate messageTemplate;
+
+        public RandomMessageGenerator(String vendorFilePath, String jargonFilePath) throws IOException {
+            List<String> vendors = FileUtil.listyFile(new File(vendorFilePath));
+            List<String> jargon = FileUtil.listyFile(new File(jargonFilePath));
+            this.messageTemplate = new MessageTemplate(vendors, jargon);
+        }
+
+        public RandomMessageGenerator(String[] vendors, String[] jargon) {
+            List<String> vendorList = new ArrayList<String>();
+            for (String v : vendors) {
+                vendorList.add(v);
+            }
+            List<String> jargonList = new ArrayList<String>();
+            for (String j : jargon) {
+                jargonList.add(j);
+            }
+            this.messageTemplate = new MessageTemplate(vendorList, jargonList);
+        }
+
+        public Message getNextRandomMessage() {
+            return messageTemplate.getNextMessage();
+        }
+    }
+
+    public static class AbstractMessageTemplate {
+
+        protected final Random random = new Random();
+
+        protected String[] positiveVerbs = new String[] { "like", "love" };
+        protected String[] negativeVerbs = new String[] { "dislike", "hate", "can't stand" };
+
+        protected String[] negativeAdjectives = new String[] { "horrible", "bad", "terrible", "OMG" };
+        protected String[] postiveAdjectives = new String[] { "good", "awesome", "amazing", "mind-blowing" };
+
+        protected String[] otherWords = new String[] { "the", "its" };
+    }
+
+    public static class MessageTemplate extends AbstractMessageTemplate {
+
+        private List<String> vendors;
+        private List<String> jargon;
+        private CharBuffer buffer;
+        private List<String> referredTopics;
+        private Message message = new Message();
+
+        public MessageTemplate(List<String> vendors, List<String> jargon) {
+            this.vendors = vendors;
+            this.jargon = jargon;
+            buffer = CharBuffer.allocate(2500);
+            referredTopics = new ArrayList<String>();
+        }
+
+        public Message getNextMessage() {
+            buffer.position(0);
+            buffer.limit(2500);
+            referredTopics.clear();
+            boolean isPositive = random.nextBoolean();
+            String[] verbArray = isPositive ? positiveVerbs : negativeVerbs;
+            String[] adjectiveArray = isPositive ? postiveAdjectives : negativeAdjectives;
+            String verb = verbArray[random.nextInt(verbArray.length)];
+            String adjective = adjectiveArray[random.nextInt(adjectiveArray.length)];
+
+            buffer.put(" ");
+            buffer.put(verb);
+            buffer.put(" ");
+            String vendor = vendors.get(random.nextInt(vendors.size()));
+            referredTopics.add(vendor);
+            buffer.append(vendor);
+            buffer.append(" ");
+            buffer.append(otherWords[random.nextInt(otherWords.length)]);
+            buffer.append(" ");
+            String jargonTerm = jargon.get(random.nextInt(jargon.size()));
+            referredTopics.add(jargonTerm);
+            buffer.append(jargonTerm);
+            buffer.append(" is ");
+            buffer.append(adjective);
+            if (random.nextBoolean()) {
+                buffer.append(isPositive ? ":)" : ":(");
+            }
+
+            buffer.flip();
+            message.reset(buffer.array(), 0, buffer.limit(), referredTopics);
+            return message;
+        }
+    }
+
+    public static class RandomUtil {
+
+        public static Random random = new Random();
+
+        public static int[] getKFromN(int k, int n) {
+            int[] result = new int[k];
+            int cnt = 0;
+            HashSet<Integer> values = new HashSet<Integer>();
+            while (cnt < k) {
+                int val = random.nextInt(n + 1);
+                if (values.contains(val)) {
+                    continue;
+                }
+
+                result[cnt++] = val;
+                values.add(val);
+            }
+            return result;
+        }
+    }
+
+    public static class FileUtil {
+
+        public static List<String> listyFile(File file) throws IOException {
+
+            BufferedReader reader = new BufferedReader(new FileReader(file));
+            String line;
+            List<String> list = new ArrayList<String>();
+            while (true) {
+                line = reader.readLine();
+                if (line == null) {
+                    break;
+                }
+                list.add(line);
+            }
+            return list;
+        }
+
+        public static FileAppender getFileAppender(String filePath, boolean createIfNotExists, boolean overwrite)
+                throws IOException {
+            return new FileAppender(filePath, createIfNotExists, overwrite);
+        }
+    }
+
+    public static class FileAppender {
+
+        private final BufferedWriter writer;
+
+        public FileAppender(String filePath, boolean createIfNotExists, boolean overwrite) throws IOException {
+            File file = new File(filePath);
+            if (!file.exists()) {
+                if (createIfNotExists) {
+                    new File(file.getParent()).mkdirs();
+                } else {
+                    throw new IOException("path " + filePath + " does not exists");
+                }
+            }
+            this.writer = new BufferedWriter(new FileWriter(file, !overwrite));
+        }
+
+        public void appendToFile(String content) throws IOException {
+            writer.append(content);
+            writer.append("\n");
+        }
+
+        public void close() throws IOException {
+            writer.close();
+        }
+    }
+
+    public static class RandomEmploymentGenerator {
+
+        private final int percentEmployed;
+        private final Random random = new Random();
+        private final RandomDateGenerator randDateGen;
+        private final List<String> organizations;
+        private Employment emp;
+
+        public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String orgListPath)
+                throws IOException {
+            this.percentEmployed = percentEmployed;
+            this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+            organizations = FileUtil.listyFile(new File(orgListPath));
+            emp = new Employment();
+        }
+
+        public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String[] orgList) {
+            this.percentEmployed = percentEmployed;
+            this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+            organizations = new ArrayList<String>();
+            for (String org : orgList) {
+                organizations.add(org);
+            }
+            emp = new Employment();
+        }
+
+        public Employment getRandomEmployment() {
+            int empployed = random.nextInt(100) + 1;
+            boolean isEmployed = false;
+            if (empployed <= percentEmployed) {
+                isEmployed = true;
+            }
+            Date startDate = randDateGen.getNextRandomDate();
+            Date endDate = null;
+            if (!isEmployed) {
+                endDate = randDateGen.getNextRecentDate(startDate);
+            }
+            String org = organizations.get(random.nextInt(organizations.size()));
+            emp.reset(org, startDate, endDate);
+            return emp;
+        }
+    }
+
+    public static class RandomLocationGenerator {
+
+        private Random random = new Random();
+
+        private final int beginLat;
+        private final int endLat;
+        private final int beginLong;
+        private final int endLong;
+
+        private Point point;
+
+        public RandomLocationGenerator(int beginLat, int endLat, int beginLong, int endLong) {
+            this.beginLat = beginLat;
+            this.endLat = endLat;
+            this.beginLong = beginLong;
+            this.endLong = endLong;
+            this.point = new Point();
+        }
+
+        public Point getRandomPoint() {
+            int latMajor = beginLat + random.nextInt(endLat - beginLat);
+            int latMinor = random.nextInt(100);
+            float latitude = latMajor + ((float) latMinor) / 100;
+
+            int longMajor = beginLong + random.nextInt(endLong - beginLong);
+            int longMinor = random.nextInt(100);
+            float longitude = longMajor + ((float) longMinor) / 100;
+
+            point.reset(latitude, longitude);
+            return point;
+        }
+
+    }
+
+    public static class PartitionConfiguration {
+
+        private final TargetPartition targetPartition;
+        private final SourcePartition sourcePartition;
+
+        public PartitionConfiguration(SourcePartition sourcePartition, TargetPartition targetPartition) {
+            this.sourcePartition = sourcePartition;
+            this.targetPartition = targetPartition;
+        }
+
+        public TargetPartition getTargetPartition() {
+            return targetPartition;
+        }
+
+        public SourcePartition getSourcePartition() {
+            return sourcePartition;
+        }
+
+    }
+
+    public static class SourcePartition {
+
+        private final String name;
+        private final String host;
+        private final String path;
+
+        public SourcePartition(String name, String host, String path) {
+            this.name = name;
+            this.host = host;
+            this.path = path;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getHost() {
+            return host;
+        }
+
+        public String getPath() {
+            return path;
+        }
+    }
+
+    public static class TargetPartition {
+        private final String name;
+        private final String host;
+        private final String path;
+        private final int fbUserKeyMin;
+        private final int fbUserKeyMax;
+        private final int twUserKeyMin;
+        private final int twUserKeyMax;
+        private final int fbMessageIdMin;
+        private final int fbMessageIdMax;
+        private final int twMessageIdMin;
+        private final int twMessageIdMax;
+        private final int commonUsers;
+
+        public TargetPartition(String partitionName, String host, String path, int fbUserKeyMin, int fbUserKeyMax,
+                int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+                int twMessageIdMax, int commonUsers) {
+            this.name = partitionName;
+            this.host = host;
+            this.path = path;
+            this.fbUserKeyMin = fbUserKeyMin;
+            this.fbUserKeyMax = fbUserKeyMax;
+            this.twUserKeyMin = twUserKeyMin;
+            this.twUserKeyMax = twUserKeyMax;
+            this.twMessageIdMin = twMessageIdMin;
+            this.twMessageIdMax = twMessageIdMax;
+            this.fbMessageIdMin = fbMessageIdMin;
+            this.fbMessageIdMax = fbMessageIdMax;
+            this.commonUsers = commonUsers;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append(name);
+            builder.append(" ");
+            builder.append(host);
+            builder.append("\n");
+            builder.append(path);
+            builder.append("\n");
+            builder.append("fbUser:key:min");
+            builder.append(fbUserKeyMin);
+
+            builder.append("\n");
+            builder.append("fbUser:key:max");
+            builder.append(fbUserKeyMax);
+
+            builder.append("\n");
+            builder.append("twUser:key:min");
+            builder.append(twUserKeyMin);
+
+            builder.append("\n");
+            builder.append("twUser:key:max");
+            builder.append(twUserKeyMax);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:min");
+            builder.append(fbMessageIdMin);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:max");
+            builder.append(fbMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:min");
+            builder.append(twMessageIdMin);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("commonUsers");
+            builder.append(commonUsers);
+
+            return new String(builder);
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getHost() {
+            return host;
+        }
+
+        public int getFbUserKeyMin() {
+            return fbUserKeyMin;
+        }
+
+        public int getFbUserKeyMax() {
+            return fbUserKeyMax;
+        }
+
+        public int getTwUserKeyMin() {
+            return twUserKeyMin;
+        }
+
+        public int getTwUserKeyMax() {
+            return twUserKeyMax;
+        }
+
+        public int getFbMessageIdMin() {
+            return fbMessageIdMin;
+        }
+
+        public int getFbMessageIdMax() {
+            return fbMessageIdMax;
+        }
+
+        public int getTwMessageIdMin() {
+            return twMessageIdMin;
+        }
+
+        public int getTwMessageIdMax() {
+            return twMessageIdMax;
+        }
+
+        public int getCommonUsers() {
+            return commonUsers;
+        }
+
+        public String getPath() {
+            return path;
+        }
+    }
+
+    public static class Employment {
+
+        private String organization;
+        private Date startDate;
+        private Date endDate;
+
+        public Employment(String organization, Date startDate, Date endDate) {
+            this.organization = organization;
+            this.startDate = startDate;
+            this.endDate = endDate;
+        }
+
+        public Employment() {
+        }
+
+        public void reset(String organization, Date startDate, Date endDate) {
+            this.organization = organization;
+            this.startDate = startDate;
+            this.endDate = endDate;
+        }
+
+        public String getOrganization() {
+            return organization;
+        }
+
+        public Date getStartDate() {
+            return startDate;
+        }
+
+        public Date getEndDate() {
+            return endDate;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder("");
+            builder.append("{");
+            builder.append("\"organization-name\":");
+            builder.append("\"" + organization + "\"");
+            builder.append(",");
+            builder.append("\"start-date\":");
+            builder.append(startDate);
+            if (endDate != null) {
+                builder.append(",");
+                builder.append("\"end-date\":");
+                builder.append(endDate);
+            }
+            builder.append("}");
+            return new String(builder);
+        }
+
+    }
+
+    public static class FacebookMessage {
+
+        private int messageId;
+        private int authorId;
+        private int inResponseTo;
+        private Point senderLocation;
+        private Message message;
+
+        public int getMessageId() {
+            return messageId;
+        }
+
+        public int getAuthorID() {
+            return authorId;
+        }
+
+        public Point getSenderLocation() {
+            return senderLocation;
+        }
+
+        public Message getMessage() {
+            return message;
+        }
+
+        public int getInResponseTo() {
+            return inResponseTo;
+        }
+
+        public FacebookMessage() {
+
+        }
+
+        public FacebookMessage(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+            this.messageId = messageId;
+            this.authorId = authorId;
+            this.inResponseTo = inResponseTo;
+            this.senderLocation = senderLocation;
+            this.message = message;
+        }
+
+        public void reset(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+            this.messageId = messageId;
+            this.authorId = authorId;
+            this.inResponseTo = inResponseTo;
+            this.senderLocation = senderLocation;
+            this.message = message;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"message-id\":");
+            builder.append(messageId);
+            builder.append(",");
+            builder.append("\"author-id\":");
+            builder.append(authorId);
+            builder.append(",");
+            builder.append("\"in-response-to\":");
+            builder.append(inResponseTo);
+            builder.append(",");
+            builder.append("\"sender-location\":");
+            builder.append(senderLocation);
+            builder.append(",");
+            builder.append("\"message\":");
+            builder.append("\"");
+            for (int i = 0; i < message.getLength(); i++) {
+                builder.append(message.charAt(i));
+            }
+            builder.append("\"");
+            builder.append("}");
+            return new String(builder);
+        }
+    }
+
+    public static class FacebookUser {
+
+        private int id;
+        private String alias;
+        private String name;
+        private String userSince;
+        private int[] friendIds;
+        private Employment employment;
+
+        public FacebookUser() {
+
+        }
+
+        public FacebookUser(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+            this.id = id;
+            this.alias = alias;
+            this.name = name;
+            this.userSince = userSince;
+            this.friendIds = friendIds;
+            this.employment = employment;
+        }
+
+        public int getId() {
+            return id;
+        }
+
+        public String getAlias() {
+            return alias;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getUserSince() {
+            return userSince;
+        }
+
+        public int[] getFriendIds() {
+            return friendIds;
+        }
+
+        public Employment getEmployment() {
+            return employment;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"id\":" + id);
+            builder.append(",");
+            builder.append("\"alias\":" + "\"" + alias + "\"");
+            builder.append(",");
+            builder.append("\"name\":" + "\"" + name + "\"");
+            builder.append(",");
+            builder.append("\"user-since\":" + userSince);
+            builder.append(",");
+            builder.append("\"friend-ids\":");
+            builder.append("{{");
+            for (int i = 0; i < friendIds.length; i++) {
+                builder.append(friendIds[i]);
+                builder.append(",");
+            }
+            if (friendIds.length > 0) {
+                builder.deleteCharAt(builder.lastIndexOf(","));
+            }
+            builder.append("}}");
+            builder.append(",");
+            builder.append("\"employment\":");
+            builder.append("[");
+            builder.append(employment.toString());
+            builder.append("]");
+            builder.append("}");
+            return builder.toString();
+        }
+
+        public void setId(int id) {
+            this.id = id;
+        }
+
+        public void setAlias(String alias) {
+            this.alias = alias;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public void setUserSince(String userSince) {
+            this.userSince = userSince;
+        }
+
+        public void setFriendIds(int[] friendIds) {
+            this.friendIds = friendIds;
+        }
+
+        public void setEmployment(Employment employment) {
+            this.employment = employment;
+        }
+
+        public void reset(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+            this.id = id;
+            this.alias = alias;
+            this.name = name;
+            this.userSince = userSince;
+            this.friendIds = friendIds;
+            this.employment = employment;
+        }
+    }
+
+    public static class TweetMessage {
+
+        private String tweetid;
+        private TwitterUser user;
+        private Point senderLocation;
+        private DateTime sendTime;
+        private List<String> referredTopics;
+        private Message messageText;
+
+        public TweetMessage() {
+
+        }
+
+        public TweetMessage(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+                List<String> referredTopics, Message messageText) {
+            this.tweetid = tweetid;
+            this.user = user;
+            this.senderLocation = senderLocation;
+            this.sendTime = sendTime;
+            this.referredTopics = referredTopics;
+            this.messageText = messageText;
+        }
+
+        public void reset(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+                List<String> referredTopics, Message messageText) {
+            this.tweetid = tweetid;
+            this.user = user;
+            this.senderLocation = senderLocation;
+            this.sendTime = sendTime;
+            this.referredTopics = referredTopics;
+            this.messageText = messageText;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"tweetid\":");
+            builder.append("\"" + tweetid + "\"");
+            builder.append(",");
+            builder.append("\"user\":");
+            builder.append(user);
+            builder.append(",");
+            builder.append("\"sender-location\":");
+            builder.append(senderLocation);
+            builder.append(",");
+            builder.append("\"send-time\":");
+            builder.append(sendTime);
+            builder.append(",");
+            builder.append("\"referred-topics\":");
+            builder.append("{{");
+            for (String topic : referredTopics) {
+                builder.append("\"" + topic + "\"");
+                builder.append(",");
+            }
+            if (referredTopics.size() > 0) {
+                builder.deleteCharAt(builder.lastIndexOf(","));
+            }
+            builder.append("}}");
+            builder.append(",");
+            builder.append("\"message-text\":");
+            builder.append("\"");
+            for (int i = 0; i < messageText.getLength(); i++) {
+                builder.append(messageText.charAt(i));
+            }
+            builder.append("\"");
+            builder.append("}");
+            return new String(builder);
+        }
+
+        public String getTweetid() {
+            return tweetid;
+        }
+
+        public void setTweetid(String tweetid) {
+            this.tweetid = tweetid;
+        }
+
+        public TwitterUser getUser() {
+            return user;
+        }
+
+        public void setUser(TwitterUser user) {
+            this.user = user;
+        }
+
+        public Point getSenderLocation() {
+            return senderLocation;
+        }
+
+        public void setSenderLocation(Point senderLocation) {
+            this.senderLocation = senderLocation;
+        }
+
+        public DateTime getSendTime() {
+            return sendTime;
+        }
+
+        public void setSendTime(DateTime sendTime) {
+            this.sendTime = sendTime;
+        }
+
+        public List<String> getReferredTopics() {
+            return referredTopics;
+        }
+
+        public void setReferredTopics(List<String> referredTopics) {
+            this.referredTopics = referredTopics;
+        }
+
+        public Message getMessageText() {
+            return messageText;
+        }
+
+        public void setMessageText(Message messageText) {
+            this.messageText = messageText;
+        }
+
+    }
+
+    public static class TwitterUser {
+
+        private String screenName;
+        private String lang = "en";
+        private int friendsCount;
+        private int statusesCount;
+        private String name;
+        private int followersCount;
+
+        public TwitterUser() {
+
+        }
+
+        public TwitterUser(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+            this.screenName = screenName;
+            this.friendsCount = friendsCount;
+            this.statusesCount = statusesCount;
+            this.name = name;
+            this.followersCount = followersCount;
+        }
+
+        public void reset(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+            this.screenName = screenName;
+            this.friendsCount = friendsCount;
+            this.statusesCount = statusesCount;
+            this.name = name;
+            this.followersCount = followersCount;
+        }
+
+        public String getScreenName() {
+            return screenName;
+        }
+
+        public int getFriendsCount() {
+            return friendsCount;
+        }
+
+        public int getStatusesCount() {
+            return statusesCount;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public int getFollowersCount() {
+            return followersCount;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"screen-name\":" + "\"" + screenName + "\"");
+            builder.append(",");
+            builder.append("\"lang\":" + "\"" + lang + "\"");
+            builder.append(",");
+            builder.append("\"friends_count\":" + friendsCount);
+            builder.append(",");
+            builder.append("\"statuses_count\":" + statusesCount);
+            builder.append(",");
+            builder.append("\"name\":" + "\"" + name + "\"");
+            builder.append(",");
+            builder.append("\"followers_count\":" + followersCount);
+            builder.append("}");
+            return builder.toString();
+        }
+
+    }
+
+    public static class DistributionHandler {
+
+        private final ZipfGenerator zipfGen;
+        private final int totalUsers;
+        private final int totalMessages;
+        private Random random = new Random();
+
+        public DistributionHandler(int totalMessages, double skew, int totalNumUsers) {
+            zipfGen = new ZipfGenerator(totalMessages, skew);
+            totalUsers = totalNumUsers;
+            this.totalMessages = totalMessages;
+        }
+
+        public int getFromDistribution(int rank) {
+            double prob = zipfGen.getProbability(rank);
+            int numMessages = (int) (prob * totalMessages);
+            return numMessages;
+        }
+
+        public static void main(String args[]) {
+            int totalMessages = 1000 * 4070;
+            double skew = 0.5;
+            int totalUsers = 4070;
+            DistributionHandler d = new DistributionHandler(totalMessages, skew, totalUsers);
+            int sum = 0;
+            for (int i = totalUsers; i >= 1; i--) {
+                float contrib = d.getFromDistribution(i);
+                sum += contrib;
+                System.out.println(i + ":" + contrib);
+            }
+
+            System.out.println("SUM" + ":" + sum);
+
+        }
+    }
+
+    public static class ZipfGenerator {
+
+        private Random rnd = new Random(System.currentTimeMillis());
+        private int size;
+        private double skew;
+        private double bottom = 0;
+
+        public ZipfGenerator(int size, double skew) {
+            this.size = size;
+            this.skew = skew;
+            for (int i = 1; i < size; i++) {
+                this.bottom += (1 / Math.pow(i, this.skew));
+            }
+        }
+
+        // the next() method returns an rank id. The frequency of returned rank
+        // ids are follows Zipf distribution.
+        public int next() {
+            int rank;
+            double friquency = 0;
+            double dice;
+            rank = rnd.nextInt(size);
+            friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+            dice = rnd.nextDouble();
+            while (!(dice < friquency)) {
+                rank = rnd.nextInt(size);
+                friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+                dice = rnd.nextDouble();
+            }
+            return rank;
+        }
+
+        // This method returns a probability that the given rank occurs.
+        public double getProbability(int rank) {
+            return (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+        }
+
+        public static void main(String[] args) throws IOException {
+            int total = (int) (3.7 * 1000 * 1000);
+            int skew = 2;
+            int numUsers = 1000 * 1000;
+            /*
+             * if (args.length != 2) { System.out.println("usage:" +
+             * "./zipf size skew"); System.exit(-1); }
+             */
+            BufferedWriter buf = new BufferedWriter(new FileWriter(new File("/tmp/zip_output")));
+            ZipfGenerator zipf = new ZipfGenerator(total, skew);
+            double sum = 0;
+            for (int i = 1; i <= numUsers; i++) {
+                double prob = zipf.getProbability(i);
+                double contribution = (double) (prob * total);
+                String contrib = i + ":" + contribution;
+                buf.write(contrib);
+                buf.write("\n");
+                System.out.println(contrib);
+                sum += contribution;
+            }
+            System.out.println("sum is :" + sum);
+        }
+    }
+
+    public static class PartitionElement implements ILibraryElement {
+        private final String name;
+        private final String host;
+        private final int fbUserKeyMin;
+        private final int fbUserKeyMax;
+        private final int twUserKeyMin;
+        private final int twUserKeyMax;
+        private final int fbMessageIdMin;
+        private final int fbMessageIdMax;
+        private final int twMessageIdMin;
+        private final int twMessageIdMax;
+
+        public PartitionElement(String partitionName, String host, int fbUserKeyMin, int fbUserKeyMax,
+                int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+                int twMessageIdMax) {
+            this.name = partitionName;
+            this.host = host;
+            this.fbUserKeyMin = fbUserKeyMin;
+            this.fbUserKeyMax = fbUserKeyMax;
+            this.twUserKeyMin = twUserKeyMax;
+            this.twUserKeyMax = twUserKeyMax;
+            this.twMessageIdMin = twMessageIdMin;
+            this.twMessageIdMax = twMessageIdMax;
+            this.fbMessageIdMin = fbMessageIdMin;
+            this.fbMessageIdMax = fbMessageIdMax;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append(name);
+            builder.append(" ");
+            builder.append(host);
+            builder.append("\n");
+            builder.append("fbUser:key:min");
+            builder.append(fbUserKeyMin);
+
+            builder.append("\n");
+            builder.append("fbUser:key:max");
+            builder.append(fbUserKeyMax);
+
+            builder.append("\n");
+            builder.append("twUser:key:min");
+            builder.append(twUserKeyMin);
+
+            builder.append("\n");
+            builder.append("twUser:key:max");
+            builder.append(twUserKeyMax);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:min");
+            builder.append(fbMessageIdMin);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:max");
+            builder.append(fbMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:min");
+            builder.append(twMessageIdMin);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twUserKeyMin);
+
+            return new String(builder);
+        }
+
+        @Override
+        public String getName() {
+            return "Partition";
+        }
+
+    }
+
+    interface ILibraryElement {
+
+        public enum ElementType {
+            PARTITION
+        }
+
+        public String getName();
+
+    }
+
+    public static class Configuration {
+
+        private final float numMB;
+        private final String unit;
+
+        private final List<SourcePartition> sourcePartitions;
+        private List<TargetPartition> targetPartitions;
+
+        public Configuration(float numMB, String unit, List<SourcePartition> partitions) throws IOException {
+            this.numMB = numMB;
+            this.unit = unit;
+            this.sourcePartitions = partitions;
+
+        }
+
+        public float getNumMB() {
+            return numMB;
+        }
+
+        public String getUnit() {
+            return unit;
+        }
+
+        public List<SourcePartition> getSourcePartitions() {
+            return sourcePartitions;
+        }
+
+        public List<TargetPartition> getTargetPartitions() {
+            return targetPartitions;
+        }
+
+        public void setTargetPartitions(List<TargetPartition> targetPartitions) {
+            this.targetPartitions = targetPartitions;
+        }
+
+    }
+
+    public static class XMLUtil {
+
+        public static void writeToXML(Configuration conf, String filePath) throws IOException,
+                ParserConfigurationException, TransformerException {
+
+            DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+
+            // root elements
+            Document doc = docBuilder.newDocument();
+            Element rootElement = doc.createElement("Partitions");
+            doc.appendChild(rootElement);
+
+            int index = 0;
+            for (TargetPartition partition : conf.getTargetPartitions()) {
+                writePartitionElement(conf.getSourcePartitions().get(index), partition, rootElement, doc);
+            }
+
+            TransformerFactory transformerFactory = TransformerFactory.newInstance();
+            Transformer transformer = transformerFactory.newTransformer();
+
+            transformer.setOutputProperty(OutputKeys.ENCODING, "utf-8");
+            transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "no");
+            transformer.setOutputProperty(OutputKeys.INDENT, "yes");
+            transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+
+            DOMSource source = new DOMSource(doc);
+            StreamResult result = new StreamResult(new File(filePath));
+
+            transformer.transform(source, result);
+
+        }
+
+        public static void writePartitionInfo(Configuration conf, String filePath) throws IOException {
+            BufferedWriter bw = new BufferedWriter(new FileWriter(filePath));
+            for (SourcePartition sp : conf.getSourcePartitions()) {
+                bw.write(sp.getHost() + ":" + sp.getName() + ":" + sp.getPath());
+                bw.write("\n");
+            }
+            bw.close();
+        }
+
+        public static Document getDocument(String filePath) throws Exception {
+            File inputFile = new File(filePath);
+            DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+            DocumentBuilder db = dbf.newDocumentBuilder();
+            Document doc = db.parse(inputFile);
+            doc.getDocumentElement().normalize();
+            return doc;
+        }
+
+        public static Configuration getConfiguration(String filePath) throws Exception {
+            Configuration conf = getConfiguration(getDocument(filePath));
+            PartitionMetrics metrics = new PartitionMetrics(conf.getNumMB(), conf.getUnit(), conf.getSourcePartitions()
+                    .size());
+            List<TargetPartition> targetPartitions = getTargetPartitions(metrics, conf.getSourcePartitions());
+            conf.setTargetPartitions(targetPartitions);
+            return conf;
+        }
+
+        public static Configuration getConfiguration(Document document) throws IOException {
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getChildNodes();
+            float size = Float.parseFloat(getStringValue((Element) nodeList, "size"));
+            String unit = getStringValue((Element) nodeList, "unit");
+            List<SourcePartition> sourcePartitions = getSourcePartitions(document);
+            return new Configuration(size, unit, sourcePartitions);
+        }
+
+        public static List<SourcePartition> getSourcePartitions(Document document) {
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getElementsByTagName("partition");
+            List<SourcePartition> sourcePartitions = new ArrayList<SourcePartition>();
+            for (int i = 0; i < nodeList.getLength(); i++) {
+                Node node = nodeList.item(i);
+                sourcePartitions.add(getSourcePartition((Element) node));
+            }
+            return sourcePartitions;
+        }
+
+        public static SourcePartition getSourcePartition(Element functionElement) {
+            String name = getStringValue(functionElement, "name");
+            String host = getStringValue(functionElement, "host");
+            String path = getStringValue(functionElement, "path");
+            SourcePartition sp = new SourcePartition(name, host, path);
+            return sp;
+        }
+
+        public static String getStringValue(Element element, String tagName) {
+            String textValue = null;
+            NodeList nl = element.getElementsByTagName(tagName);
+            if (nl != null && nl.getLength() > 0) {
+                Element el = (Element) nl.item(0);
+                textValue = el.getFirstChild().getNodeValue();
+            }
+            return textValue;
+        }
+
+        public static PartitionConfiguration getPartitionConfiguration(String filePath, String partitionName)
+                throws Exception {
+            PartitionConfiguration pconf = getPartitionConfigurations(getDocument(filePath), partitionName);
+            return pconf;
+        }
+
+        public static PartitionConfiguration getPartitionConfigurations(Document document, String partitionName)
+                throws IOException {
+
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getElementsByTagName("Partition");
+
+            for (int i = 0; i < nodeList.getLength(); i++) {
+                Node node = nodeList.item(i);
+                Element nodeElement = (Element) node;
+                String name = getStringValue(nodeElement, "name");
+                if (!name.equalsIgnoreCase(partitionName)) {
+                    continue;
+                }
+                String host = getStringValue(nodeElement, "host");
+                String path = getStringValue(nodeElement, "path");
+
+                String fbUserKeyMin = getStringValue(nodeElement, "fbUserKeyMin");
+                String fbUserKeyMax = getStringValue(nodeElement, "fbUserKeyMax");
+                String twUserKeyMin = getStringValue(nodeElement, "twUserKeyMin");
+                String twUserKeyMax = getStringValue(nodeElement, "twUserKeyMax");
+                String fbMessageKeyMin = getStringValue(nodeElement, "fbMessageKeyMin");
+
+                String fbMessageKeyMax = getStringValue(nodeElement, "fbMessageKeyMax");
+                String twMessageKeyMin = getStringValue(nodeElement, "twMessageKeyMin");
+                String twMessageKeyMax = getStringValue(nodeElement, "twMessageKeyMax");
+                String numCommonUsers = getStringValue(nodeElement, "numCommonUsers");
+
+                SourcePartition sp = new SourcePartition(name, host, path);
+
+                TargetPartition tp = new TargetPartition(partitionName, host, path, Integer.parseInt(fbUserKeyMin),
+                        Integer.parseInt(fbUserKeyMax), Integer.parseInt(twUserKeyMin), Integer.parseInt(twUserKeyMax),
+                        Integer.parseInt(fbMessageKeyMin), Integer.parseInt(fbMessageKeyMax),
+                        Integer.parseInt(twMessageKeyMin), Integer.parseInt(twMessageKeyMax),
+                        Integer.parseInt(numCommonUsers));
+                PartitionConfiguration pc = new PartitionConfiguration(sp, tp);
+                return pc;
+            }
+            return null;
+        }
+
+        public static List<TargetPartition> getTargetPartitions(PartitionMetrics metrics,
+                List<SourcePartition> sourcePartitions) {
+            List<TargetPartition> partitions = new ArrayList<TargetPartition>();
+            int fbUserKeyMin = 1;
+            int twUserKeyMin = 1;
+            int fbMessageIdMin = 1;
+            int twMessageIdMin = 1;
+
+            for (SourcePartition sp : sourcePartitions) {
+                int fbUserKeyMax = fbUserKeyMin + metrics.getFbOnlyUsers() + metrics.getCommonUsers() - 1;
+                int twUserKeyMax = twUserKeyMin + metrics.getTwitterOnlyUsers() + metrics.getCommonUsers() - 1;
+
+                int fbMessageIdMax = fbMessageIdMin + metrics.getFbMessages() - 1;
+                int twMessageIdMax = twMessageIdMin + metrics.getTwMessages() - 1;
+                TargetPartition pe = new TargetPartition(sp.getName(), sp.getHost(), sp.getPath(), fbUserKeyMin,
+                        fbUserKeyMax, twUserKeyMin, twUserKeyMax, fbMessageIdMin, fbMessageIdMax, twMessageIdMin,
+                        twMessageIdMax, metrics.getCommonUsers());
+                partitions.add(pe);
+
+                fbUserKeyMin = fbUserKeyMax + 1;
+                twUserKeyMin = twUserKeyMax + 1;
+
+                fbMessageIdMin = fbMessageIdMax + 1;
+                twMessageIdMin = twMessageIdMax + 1;
+            }
+
+            return partitions;
+        }
+
+        public static void writePartitionElement(SourcePartition sourcePartition, TargetPartition partition,
+                Element rootElement, Document doc) {
+            // staff elements
+            Element pe = doc.createElement("Partition");
+            rootElement.appendChild(pe);
+
+            // name element
+            Element name = doc.createElement("name");
+            name.appendChild(doc.createTextNode("" + partition.getName()));
+            pe.appendChild(name);
+
+            // host element
+            Element host = doc.createElement("host");
+            host.appendChild(doc.createTextNode("" + partition.getHost()));
+            pe.appendChild(host);
+
+            // path element
+            Element path = doc.createElement("path");
+            path.appendChild(doc.createTextNode("" + partition.getPath()));
+            pe.appendChild(path);
+
+            // fbUserKeyMin element
+            Element fbUserKeyMin = doc.createElement("fbUserKeyMin");
+            fbUserKeyMin.appendChild(doc.createTextNode("" + partition.getFbUserKeyMin()));
+            pe.appendChild(fbUserKeyMin);
+
+            // fbUserKeyMax element
+            Element fbUserKeyMax = doc.createElement("fbUserKeyMax");
+            fbUserKeyMax.appendChild(doc.createTextNode("" + partition.getFbUserKeyMax()));
+            pe.appendChild(fbUserKeyMax);
+
+            // twUserKeyMin element
+            Element twUserKeyMin = doc.createElement("twUserKeyMin");
+            twUserKeyMin.appendChild(doc.createTextNode("" + partition.getTwUserKeyMin()));
+            pe.appendChild(twUserKeyMin);
+
+            // twUserKeyMax element
+            Element twUserKeyMax = doc.createElement("twUserKeyMax");
+            twUserKeyMax.appendChild(doc.createTextNode("" + partition.getTwUserKeyMax()));
+            pe.appendChild(twUserKeyMax);
+
+            // fbMessgeKeyMin element
+            Element fbMessageKeyMin = doc.createElement("fbMessageKeyMin");
+            fbMessageKeyMin.appendChild(doc.createTextNode("" + partition.getFbMessageIdMin()));
+            pe.appendChild(fbMessageKeyMin);
+
+            // fbMessgeKeyMin element
+            Element fbMessageKeyMax = doc.createElement("fbMessageKeyMax");
+            fbMessageKeyMax.appendChild(doc.createTextNode("" + partition.getFbMessageIdMax()));
+            pe.appendChild(fbMessageKeyMax);
+
+            // twMessgeKeyMin element
+            Element twMessageKeyMin = doc.createElement("twMessageKeyMin");
+            twMessageKeyMin.appendChild(doc.createTextNode("" + partition.getTwMessageIdMin()));
+            pe.appendChild(twMessageKeyMin);
+
+            // twMessgeKeyMin element
+            Element twMessageKeyMax = doc.createElement("twMessageKeyMax");
+            twMessageKeyMax.appendChild(doc.createTextNode("" + partition.getTwMessageIdMax()));
+            pe.appendChild(twMessageKeyMax);
+
+            // twMessgeKeyMin element
+            Element numCommonUsers = doc.createElement("numCommonUsers");
+            numCommonUsers.appendChild(doc.createTextNode("" + partition.getCommonUsers()));
+            pe.appendChild(numCommonUsers);
+
+        }
+
+        public static void main(String args[]) throws Exception {
+            String confFile = "/Users/rgrove1/work/research/asterix/icde/data-gen/conf/conf.xml";
+            String outputPath = "/Users/rgrove1/work/research/asterix/icde/data-gen/output/conf-output.xml";
+            Configuration conf = getConfiguration(confFile);
+            writeToXML(conf, outputPath);
+        }
+
+    }
+
+    public static class Date {
+
+        private int day;
+        private int month;
+        private int year;
+
+        public Date(int month, int day, int year) {
+            this.month = month;
+            this.day = day;
+            this.year = year;
+        }
+
+        public void reset(int month, int day, int year) {
+            this.month = month;
+            this.day = day;
+            this.year = year;
+        }
+
+        public int getDay() {
+            return day;
+        }
+
+        public int getMonth() {
+            return month;
+        }
+
+        public int getYear() {
+            return year;
+        }
+
+        public Date() {
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("date");
+            builder.append("(\"");
+            builder.append(year);
+            builder.append("-");
+            builder.append(month < 10 ? "0" + month : "" + month);
+            builder.append("-");
+            builder.append(day < 10 ? "0" + day : "" + day);
+            builder.append("\")");
+            return builder.toString();
+        }
+
+        public void setDay(int day) {
+            this.day = day;
+        }
+
+        public void setMonth(int month) {
+            this.month = month;
+        }
+
+        public void setYear(int year) {
+            this.year = year;
+        }
+    }
+
+    public static class PartitionMetrics {
+
+        private final int fbMessages;
+        private final int twMessages;
+
+        private final int fbOnlyUsers;
+        private final int twitterOnlyUsers;
+        private final int commonUsers;
+
+        public PartitionMetrics(float number, String unit, int numPartitions) throws IOException {
+
+            int factor = 0;
+            if (unit.equalsIgnoreCase("MB")) {
+                factor = 1024 * 1024;
+            } else if (unit.equalsIgnoreCase("GB")) {
+                factor = 1024 * 1024 * 1024;
+            } else if (unit.equalsIgnoreCase("TB")) {
+                factor = 1024 * 1024 * 1024 * 1024;
+            } else
+                throw new IOException("Invalid unit:" + unit);
+
+            fbMessages = (int) (((number * factor * 0.80) / 258) / numPartitions);
+            twMessages = (int) (fbMessages * 1.1 / 0.35);
+
+            fbOnlyUsers = (int) ((number * factor * 0.20 * 0.0043)) / numPartitions;
+            twitterOnlyUsers = (int) (0.25 * fbOnlyUsers);
+            commonUsers = (int) (0.1 * fbOnlyUsers);
+        }
+
+        public int getFbMessages() {
+            return fbMessages;
+        }
+
+        public int getTwMessages() {
+            return twMessages;
+        }
+
+        public int getFbOnlyUsers() {
+            return fbOnlyUsers;
+        }
+
+        public int getTwitterOnlyUsers() {
+            return twitterOnlyUsers;
+        }
+
+        public int getCommonUsers() {
+            return commonUsers;
+        }
+
+    }
+
+    public static String[] lastNames = { "Hoopengarner", "Harrow", "Gardner", "Blyant", "Best", "Buttermore", "Gronko",
+            "Mayers", "Countryman", "Neely", "Ruhl", "Taggart", "Bash", "Cason", "Hil", "Zalack", "Mingle", "Carr",
+            "Rohtin", "Wardle", "Pullman", "Wire", "Kellogg", "Hiles", "Keppel", "Bratton", "Sutton", "Wickes",
+            "Muller", "Friedline", "Llora", "Elizabeth", "Anderson", "Gaskins", "Rifler", "Vinsant", "Stanfield",
+            "Black", "Guest", "Hujsak", "Carter", "Weidemann", "Hays", "Patton", "Hayhurst", "Paynter", "Cressman",
+            "Fiddler", "Evans", "Sherlock", "Woodworth", "Jackson", "Bloise", "Schneider", "Ring", "Kepplinger",
+            "James", "Moon", "Bennett", "Bashline", "Ryals", "Zeal", "Christman", "Milliron", "Nash", "Ewing", "Camp",
+            "Mason", "Richardson", "Bowchiew", "Hahn", "Wilson", "Wood", "Toyley", "Williamson", "Lafortune", "Errett",
+            "Saltser", "Hirleman", "Brindle", "Newbiggin", "Ulery", "Lambert", "Shick", "Kuster", "Moore", "Finck",
+            "Powell", "Jolce", "Townsend", "Sauter", "Cowher", "Wolfe", "Cavalet", "Porter", "Laborde", "Ballou",
+            "Murray", "Stoddard", "Pycroft", "Milne", "King", "Todd", "Staymates", "Hall", "Romanoff", "Keilbach",
+            "Sandford", "Hamilton", "Fye", "Kline", "Weeks", "Mcelroy", "Mccullough", "Bryant", "Hill", "Moore",
+            "Ledgerwood", "Prevatt", "Eckert", "Read", "Hastings", "Doverspike", "Allshouse", "Bryan", "Mccallum",
+            "Lombardi", "Mckendrick", "Cattley", "Barkley", "Steiner", "Finlay", "Priebe", "Armitage", "Hall", "Elder",
+            "Erskine", "Hatcher", "Walker", "Pearsall", "Dunkle", "Haile", "Adams", "Miller", "Newbern", "Basinger",
+            "Fuhrer", "Brinigh", "Mench", "Blackburn", "Bastion", "Mccune", "Bridger", "Hynes", "Quinn", "Courtney",
+            "Geddinge", "Field", "Seelig", "Cable", "Earhart", "Harshman", "Roby", "Beals", "Berry", "Reed", "Hector",
+            "Pittman", "Haverrman", "Kalp", "Briner", "Joghs", "Cowart", "Close", "Wynne", "Harden", "Weldy",
+            "Stephenson", "Hildyard", "Moberly", "Wells", "Mackendoerfer", "Fisher", "Oppie", "Oneal", "Churchill",
+            "Keister", "Alice", "Tavoularis", "Fisher", "Hair", "Burns", "Veith", "Wile", "Fuller", "Fields", "Clark",
+            "Randolph", "Stone", "Mcclymonds", "Holtzer", "Donkin", "Wilkinson", "Rosensteel", "Albright", "Stahl",
+            "Fox", "Kadel", "Houser", "Hanseu", "Henderson", "Davis", "Bicknell", "Swain", "Mercer", "Holdeman",
+            "Enderly", "Caesar", "Margaret", "Munshower", "Elless", "Lucy", "Feufer", "Schofield", "Graham",
+            "Blatenberger", "Benford", "Akers", "Campbell", "Ann", "Sadley", "Ling", "Gongaware", "Schmidt", "Endsley",
+            "Groah", "Flanders", "Reichard", "Lowstetter", "Sandblom", "Griffis", "Basmanoff", "Coveney", "Hawker",
+            "Archibald", "Hutton", "Barnes", "Diegel", "Raybould", "Focell", "Breitenstein", "Murray", "Chauvin",
+            "Busk", "Pheleps", "Teagarden", "Northey", "Baumgartner", "Fleming", "Harris", "Parkinson", "Carpenter",
+            "Whirlow", "Bonner", "Wortman", "Rogers", "Scott", "Lowe", "Mckee", "Huston", "Bullard", "Throckmorton",
+            "Rummel", "Mathews", "Dull", "Saline", "Tue", "Woolery", "Lalty", "Schrader", "Ramsey", "Eisenmann",
+            "Philbrick", "Sybilla", "Wallace", "Fonblanque", "Paul", "Orbell", "Higgens", "Casteel", "Franks",
+            "Demuth", "Eisenman", "Hay", "Robinson", "Fischer", "Hincken", "Wylie", "Leichter", "Bousum",
+            "Littlefield", "Mcdonald", "Greif", "Rhodes", "Wall", "Steele", "Baldwin", "Smith", "Stewart", "Schere",
+            "Mary", "Aultman", "Emrick", "Guess", "Mitchell", "Painter", "Aft", "Hasely", "Weldi", "Loewentsein",
+            "Poorbaugh", "Kepple", "Noton", "Judge", "Jackson", "Style", "Adcock", "Diller", "Marriman", "Johnston",
+            "Children", "Monahan", "Ehret", "Shaw", "Congdon", "Pinney", "Millard", "Crissman", "Tanner", "Rose",
+            "Knisely", "Cypret", "Sommer", "Poehl", "Hardie", "Bender", "Overholt", "Gottwine", "Beach", "Leslie",
+            "Trevithick", "Langston", "Magor", "Shotts", "Howe", "Hunter", "Cross", "Kistler", "Dealtry", "Christner",
+            "Pennington", "Thorley", "Eckhardstein", "Van", "Stroh", "Stough", "Stall", "Beedell", "Shea", "Garland",
+            "Mays", "Pritchard", "Frankenberger", "Rowley", "Lane", "Baum", "Alliman", "Park", "Jardine", "Butler",
+            "Cherry", "Kooser", "Baxter", "Billimek", "Downing", "Hurst", "Wood", "Baird", "Watkins", "Edwards",
+            "Kemerer", "Harding", "Owens", "Eiford", "Keener", "Garneis", "Fiscina", "Mang", "Draudy", "Mills",
+            "Gibson", "Reese", "Todd", "Ramos", "Levett", "Wilks", "Ward", "Mosser", "Dunlap", "Kifer", "Christopher",
+            "Ashbaugh", "Wynter", "Rawls", "Cribbs", "Haynes", "Thigpen", "Schreckengost", "Bishop", "Linton",
+            "Chapman", "James", "Jerome", "Hook", "Omara", "Houston", "Maclagan", "Sandys", "Pickering", "Blois",
+            "Dickson", "Kemble", "Duncan", "Woodward", "Southern", "Henley", "Treeby", "Cram", "Elsas", "Driggers",
+            "Warrick", "Overstreet", "Hindman", "Buck", "Sulyard", "Wentzel", "Swink", "Butt", "Schaeffer",
+            "Hoffhants", "Bould", "Willcox", "Lotherington", "Bagley", "Graff", "White", "Wheeler", "Sloan",
+            "Rodacker", "Hanford", "Jowers", "Kunkle", "Cass", "Powers", "Gilman", "Mcmichaels", "Hobbs", "Herndon",
+            "Prescott", "Smail", "Mcdonald", "Biery", "Orner", "Richards", "Mueller", "Isaman", "Bruxner", "Goodman",
+            "Barth", "Turzanski", "Vorrasi", "Stainforth", "Nehling", "Rahl", "Erschoff", "Greene", "Mckinnon",
+            "Reade", "Smith", "Pery", "Roose", "Greenwood", "Weisgarber", "Curry", "Holts", "Zadovsky", "Parrish",
+            "Putnam", "Munson", "Mcindoe", "Nickolson", "Brooks", "Bollinger", "Stroble", "Siegrist", "Fulton",
+            "Tomey", "Zoucks", "Roberts", "Otis", "Clarke", "Easter", "Johnson", "Fylbrigg", "Taylor", "Swartzbaugh",
+            "Weinstein", "Gadow", "Sayre", "Marcotte", "Wise", "Atweeke", "Mcfall", "Napier", "Eisenhart", "Canham",
+            "Sealis", "Baughman", "Gertraht", "Losey", "Laurence", "Eva", "Pershing", "Kern", "Pirl", "Rega",
+            "Sanborn", "Kanaga", "Sanders", "Anderson", "Dickinson", "Osteen", "Gettemy", "Crom", "Snyder", "Reed",
+            "Laurenzi", "Riggle", "Tillson", "Fowler", "Raub", "Jenner", "Koepple", "Soames", "Goldvogel", "Dimsdale",
+            "Zimmer", "Giesen", "Baker", "Beail", "Mortland", "Bard", "Sanner", "Knopsnider", "Jenkins", "Bailey",
+            "Werner", "Barrett", "Faust", "Agg", "Tomlinson", "Williams", "Little", "Greenawalt", "Wells", "Wilkins",
+            "Gisiko", "Bauerle", "Harrold", "Prechtl", "Polson", "Faast", "Winton", "Garneys", "Peters", "Potter",
+            "Porter", "Tennant", "Eve", "Dugger", "Jones", "Burch", "Cowper", "Whittier" };
+
+    public static String[] firstNames = { "Albert", "Jacquelin", "Dona", "Alia", "Mayme", "Genoveva", "Emma", "Lena",
+            "Melody", "Vilma", "Katelyn", "Jeremy", "Coral", "Leann", "Lita", "Gilda", "Kayla", "Alvina", "Maranda",
+            "Verlie", "Khadijah", "Karey", "Patrice", "Kallie", "Corey", "Mollie", "Daisy", "Melanie", "Sarita",
+            "Nichole", "Pricilla", "Terresa", "Berneice", "Arianne", "Brianne", "Lavinia", "Ulrike", "Lesha", "Adell",
+            "Ardelle", "Marisha", "Laquita", "Karyl", "Maryjane", "Kendall", "Isobel", "Raeann", "Heike", "Barbera",
+            "Norman", "Yasmine", "Nevada", "Mariam", "Edith", "Eugena", "Lovie", "Maren", "Bennie", "Lennie", "Tamera",
+            "Crystal", "Randi", "Anamaria", "Chantal", "Jesenia", "Avis", "Shela", "Randy", "Laurena", "Sharron",
+            "Christiane", "Lorie", "Mario", "Elizabeth", "Reina", "Adria", "Lakisha", "Brittni", "Azzie", "Dori",
+            "Shaneka", "Asuncion", "Katheryn", "Laurice", "Sharita", "Krystal", "Reva", "Inger", "Alpha", "Makeda",
+            "Anabel", "Loni", "Tiara", "Meda", "Latashia", "Leola", "Chin", "Daisey", "Ivory", "Amalia", "Logan",
+            "Tyler", "Kyong", "Carolann", "Maryetta", "Eufemia", "Anya", "Doreatha", "Lorna", "Rutha", "Ehtel",
+            "Debbie", "Chassidy", "Sang", "Christa", "Lottie", "Chun", "Karine", "Peggie", "Amina", "Melany", "Alayna",
+            "Scott", "Romana", "Naomi", "Christiana", "Salena", "Taunya", "Mitsue", "Regina", "Chelsie", "Charity",
+            "Dacia", "Aletha", "Latosha", "Lia", "Tamica", "Chery", "Bianca", "Shu", "Georgianne", "Myriam", "Austin",
+            "Wan", "Mallory", "Jana", "Georgie", "Jenell", "Kori", "Vicki", "Delfina", "June", "Mellisa", "Catherina",
+            "Claudie", "Tynisha", "Dayle", "Enriqueta", "Belen", "Pia", "Sarai", "Rosy", "Renay", "Kacie", "Frieda",
+            "Cayla", "Elissa", "Claribel", "Sabina", "Mackenzie", "Raina", "Cira", "Mitzie", "Aubrey", "Serafina",
+            "Maria", "Katharine", "Esperanza", "Sung", "Daria", "Billye", "Stefanie", "Kasha", "Holly", "Suzanne",
+            "Inga", "Flora", "Andria", "Genevie", "Eladia", "Janet", "Erline", "Renna", "Georgeanna", "Delorse",
+            "Elnora", "Rudy", "Rima", "Leanora", "Letisha", "Love", "Alverta", "Pinkie", "Domonique", "Jeannie",
+            "Jose", "Jacqueline", "Tara", "Lily", "Erna", "Tennille", "Galina", "Tamala", "Kirby", "Nichelle",
+            "Myesha", "Farah", "Santa", "Ludie", "Kenia", "Yee", "Micheline", "Maryann", "Elaina", "Ethelyn",
+            "Emmaline", "Shanell", "Marina", "Nila", "Alane", "Shakira", "Dorris", "Belinda", "Elois", "Barbie",
+            "Carita", "Gisela", "Lura", "Fransisca", "Helga", "Peg", "Leonarda", "Earlie", "Deetta", "Jacquetta",
+            "Blossom", "Kayleigh", "Deloras", "Keshia", "Christinia", "Dulce", "Bernie", "Sheba", "Lashanda", "Tula",
+            "Claretta", "Kary", "Jeanette", "Lupita", "Lenora", "Hisako", "Sherise", "Glynda", "Adela", "Chia",
+            "Sudie", "Mindy", "Caroyln", "Lindsey", "Xiomara", "Mercedes", "Onie", "Loan", "Alexis", "Tommie",
+            "Donette", "Monica", "Soo", "Camellia", "Lavera", "Valery", "Ariana", "Sophia", "Loris", "Ginette",
+            "Marielle", "Tari", "Julissa", "Alesia", "Suzanna", "Emelda", "Erin", "Ladawn", "Sherilyn", "Candice",
+            "Nereida", "Fairy", "Carl", "Joel", "Marilee", "Gracia", "Cordie", "So", "Shanita", "Drew", "Cassie",
+            "Sherie", "Marget", "Norma", "Delois", "Debera", "Chanelle", "Catarina", "Aracely", "Carlene", "Tricia",
+            "Aleen", "Katharina", "Marguerita", "Guadalupe", "Margorie", "Mandie", "Kathe", "Chong", "Sage", "Faith",
+            "Maryrose", "Stephany", "Ivy", "Pauline", "Susie", "Cristen", "Jenifer", "Annette", "Debi", "Karmen",
+            "Luci", "Shayla", "Hope", "Ocie", "Sharie", "Tami", "Breana", "Kerry", "Rubye", "Lashay", "Sondra",
+            "Katrice", "Brunilda", "Cortney", "Yan", "Zenobia", "Penni", "Addie", "Lavona", "Noel", "Anika",
+            "Herlinda", "Valencia", "Bunny", "Tory", "Victoria", "Carrie", "Mikaela", "Wilhelmina", "Chung",
+            "Hortencia", "Gerda", "Wen", "Ilana", "Sibyl", "Candida", "Victorina", "Chantell", "Casie", "Emeline",
+            "Dominica", "Cecila", "Delora", "Miesha", "Nova", "Sally", "Ronald", "Charlette", "Francisca", "Mina",
+            "Jenna", "Loraine", "Felisa", "Lulu", "Page", "Lyda", "Babara", "Flor", "Walter", "Chan", "Sherika",
+            "Kala", "Luna", "Vada", "Syreeta", "Slyvia", "Karin", "Renata", "Robbi", "Glenda", "Delsie", "Lizzie",
+            "Genia", "Caitlin", "Bebe", "Cory", "Sam", "Leslee", "Elva", "Caren", "Kasie", "Leticia", "Shannan",
+            "Vickey", "Sandie", "Kyle", "Chang", "Terrilyn", "Sandra", "Elida", "Marketta", "Elsy", "Tu", "Carman",
+            "Ashlie", "Vernia", "Albertine", "Vivian", "Elba", "Bong", "Margy", "Janetta", "Xiao", "Teofila", "Danyel",
+            "Nickole", "Aleisha", "Tera", "Cleotilde", "Dara", "Paulita", "Isela", "Maricela", "Rozella", "Marivel",
+            "Aurora", "Melissa", "Carylon", "Delinda", "Marvella", "Candelaria", "Deidre", "Tawanna", "Myrtie",
+            "Milagro", "Emilie", "Coretta", "Ivette", "Suzann", "Ammie", "Lucina", "Lory", "Tena", "Eleanor",
+            "Cherlyn", "Tiana", "Brianna", "Myra", "Flo", "Carisa", "Kandi", "Erlinda", "Jacqulyn", "Fermina", "Riva",
+            "Palmira", "Lindsay", "Annmarie", "Tamiko", "Carline", "Amelia", "Quiana", "Lashawna", "Veola", "Belva",
+            "Marsha", "Verlene", "Alex", "Leisha", "Camila", "Mirtha", "Melva", "Lina", "Arla", "Cythia", "Towanda",
+            "Aracelis", "Tasia", "Aurore", "Trinity", "Bernadine", "Farrah", "Deneen", "Ines", "Betty", "Lorretta",
+            "Dorethea", "Hertha", "Rochelle", "Juli", "Shenika", "Yung", "Lavon", "Deeanna", "Nakia", "Lynnette",
+            "Dinorah", "Nery", "Elene", "Carolee", "Mira", "Franchesca", "Lavonda", "Leida", "Paulette", "Dorine",
+            "Allegra", "Keva", "Jeffrey", "Bernardina", "Maryln", "Yoko", "Faviola", "Jayne", "Lucilla", "Charita",
+            "Ewa", "Ella", "Maggie", "Ivey", "Bettie", "Jerri", "Marni", "Bibi", "Sabrina", "Sarah", "Marleen",
+            "Katherin", "Remona", "Jamika", "Antonina", "Oliva", "Lajuana", "Fonda", "Sigrid", "Yael", "Billi",
+            "Verona", "Arminda", "Mirna", "Tesha", "Katheleen", "Bonita", "Kamilah", "Patrica", "Julio", "Shaina",
+            "Mellie", "Denyse", "Deandrea", "Alena", "Meg", "Kizzie", "Krissy", "Karly", "Alleen", "Yahaira", "Lucie",
+            "Karena", "Elaine", "Eloise", "Buena", "Marianela", "Renee", "Nan", "Carolynn", "Windy", "Avril", "Jane",
+            "Vida", "Thea", "Marvel", "Rosaline", "Tifany", "Robena", "Azucena", "Carlota", "Mindi", "Andera", "Jenny",
+            "Courtney", "Lyndsey", "Willette", "Kristie", "Shaniqua", "Tabatha", "Ngoc", "Una", "Marlena", "Louetta",
+            "Vernie", "Brandy", "Jacquelyne", "Jenelle", "Elna", "Erminia", "Ida", "Audie", "Louis", "Marisol",
+            "Shawana", "Harriette", "Karol", "Kitty", "Esmeralda", "Vivienne", "Eloisa", "Iris", "Jeanice", "Cammie",
+            "Jacinda", "Shena", "Floy", "Theda", "Lourdes", "Jayna", "Marg", "Kati", "Tanna", "Rosalyn", "Maxima",
+            "Soon", "Angelika", "Shonna", "Merle", "Kassandra", "Deedee", "Heidi", "Marti", "Renae", "Arleen",
+            "Alfredia", "Jewell", "Carley", "Pennie", "Corina", "Tonisha", "Natividad", "Lilliana", "Darcie", "Shawna",
+            "Angel", "Piedad", "Josefa", "Rebbeca", "Natacha", "Nenita", "Petrina", "Carmon", "Chasidy", "Temika",
+            "Dennise", "Renetta", "Augusta", "Shirlee", "Valeri", "Casimira", "Janay", "Berniece", "Deborah", "Yaeko",
+            "Mimi", "Digna", "Irish", "Cher", "Yong", "Lucila", "Jimmie", "Junko", "Lezlie", "Waneta", "Sandee",
+            "Marquita", "Eura", "Freeda", "Annabell", "Laree", "Jaye", "Wendy", "Toshia", "Kylee", "Aleta", "Emiko",
+            "Clorinda", "Sixta", "Audrea", "Juanita", "Birdie", "Reita", "Latanya", "Nia", "Leora", "Laurine",
+            "Krysten", "Jerrie", "Chantel", "Ira", "Sena", "Andre", "Jann", "Marla", "Precious", "Katy", "Gabrielle",
+            "Yvette", "Brook", "Shirlene", "Eldora", "Laura", "Milda", "Euna", "Jettie", "Debora", "Lise", "Edythe",
+            "Leandra", "Shandi", "Araceli", "Johanne", "Nieves", "Denese", "Carmelita", "Nohemi", "Annice", "Natalie",
+            "Yolande", "Jeffie", "Vashti", "Vickie", "Obdulia", "Youlanda", "Lupe", "Tomoko", "Monserrate", "Domitila",
+            "Etsuko", "Adrienne", "Lakesha", "Melissia", "Odessa", "Meagan", "Veronika", "Jolyn", "Isabelle", "Leah",
+            "Rhiannon", "Gianna", "Audra", "Sommer", "Renate", "Perla", "Thao", "Myong", "Lavette", "Mark", "Emilia",
+            "Ariane", "Karl", "Dorie", "Jacquie", "Mia", "Malka", "Shenita", "Tashina", "Christine", "Cherri", "Roni",
+            "Fran", "Mildred", "Sara", "Clarissa", "Fredia", "Elease", "Samuel", "Earlene", "Vernita", "Mae", "Concha",
+            "Renea", "Tamekia", "Hye", "Ingeborg", "Tessa", "Kelly", "Kristin", "Tam", "Sacha", "Kanisha", "Jillian",
+            "Tiffanie", "Ashlee", "Madelyn", "Donya", "Clementine", "Mickie", "My", "Zena", "Terrie", "Samatha",
+            "Gertie", "Tarra", "Natalia", "Sharlene", "Evie", "Shalon", "Rosalee", "Numbers", "Jodi", "Hattie",
+            "Naoma", "Valene", "Whitley", "Claude", "Alline", "Jeanne", "Camie", "Maragret", "Viola", "Kris", "Marlo",
+            "Arcelia", "Shari", "Jalisa", "Corrie", "Eleonor", "Angelyn", "Merry", "Lauren", "Melita", "Gita",
+            "Elenor", "Aurelia", "Janae", "Lyndia", "Margeret", "Shawanda", "Rolande", "Shirl", "Madeleine", "Celinda",
+            "Jaleesa", "Shemika", "Joye", "Tisa", "Trudie", "Kathrine", "Clarita", "Dinah", "Georgia", "Antoinette",
+            "Janis", "Suzette", "Sherri", "Herta", "Arie", "Hedy", "Cassi", "Audrie", "Caryl", "Jazmine", "Jessica",
+            "Beverly", "Elizbeth", "Marylee", "Londa", "Fredericka", "Argelia", "Nana", "Donnette", "Damaris",
+            "Hailey", "Jamee", "Kathlene", "Glayds", "Lydia", "Apryl", "Verla", "Adam", "Concepcion", "Zelda",
+            "Shonta", "Vernice", "Detra", "Meghann", "Sherley", "Sheri", "Kiyoko", "Margarita", "Adaline", "Mariela",
+            "Velda", "Ailene", "Juliane", "Aiko", "Edyth", "Cecelia", "Shavon", "Florance", "Madeline", "Rheba",
+            "Deann", "Ignacia", "Odelia", "Heide", "Mica", "Jennette", "Maricruz", "Ouida", "Darcy", "Laure",
+            "Justina", "Amada", "Laine", "Cruz", "Sunny", "Francene", "Roxanna", "Nam", "Nancie", "Deanna", "Letty",
+            "Britni", "Kazuko", "Lacresha", "Simon", "Caleb", "Milton", "Colton", "Travis", "Miles", "Jonathan",
+            "Logan", "Rolf", "Emilio", "Roberto", "Marcus", "Tim", "Delmar", "Devon", "Kurt", "Edward", "Jeffrey",
+            "Elvis", "Alfonso", "Blair", "Wm", "Sheldon", "Leonel", "Michal", "Federico", "Jacques", "Leslie",
+            "Augustine", "Hugh", "Brant", "Hong", "Sal", "Modesto", "Curtis", "Jefferey", "Adam", "John", "Glenn",
+            "Vance", "Alejandro", "Refugio", "Lucio", "Demarcus", "Chang", "Huey", "Neville", "Preston", "Bert",
+            "Abram", "Foster", "Jamison", "Kirby", "Erich", "Manual", "Dustin", "Derrick", "Donnie", "Jospeh", "Chris",
+            "Josue", "Stevie", "Russ", "Stanley", "Nicolas", "Samuel", "Waldo", "Jake", "Max", "Ernest", "Reinaldo",
+            "Rene", "Gale", "Morris", "Nathan", "Maximo", "Courtney", "Theodore", "Octavio", "Otha", "Delmer",
+            "Graham", "Dean", "Lowell", "Myles", "Colby", "Boyd", "Adolph", "Jarrod", "Nick", "Mark", "Clinton", "Kim",
+            "Sonny", "Dalton", "Tyler", "Jody", "Orville", "Luther", "Rubin", "Hollis", "Rashad", "Barton", "Vicente",
+            "Ted", "Rick", "Carmine", "Clifton", "Gayle", "Christopher", "Jessie", "Bradley", "Clay", "Theo", "Josh",
+            "Mitchell", "Boyce", "Chung", "Eugenio", "August", "Norbert", "Sammie", "Jerry", "Adan", "Edmundo",
+            "Homer", "Hilton", "Tod", "Kirk", "Emmett", "Milan", "Quincy", "Jewell", "Herb", "Steve", "Carmen",
+            "Bobby", "Odis", "Daron", "Jeremy", "Carl", "Hunter", "Tuan", "Thurman", "Asa", "Brenton", "Shane",
+            "Donny", "Andreas", "Teddy", "Dario", "Cyril", "Hoyt", "Teodoro", "Vincenzo", "Hilario", "Daren",
+            "Agustin", "Marquis", "Ezekiel", "Brendan", "Johnson", "Alden", "Richie", "Granville", "Chad", "Joseph",
+            "Lamont", "Jordon", "Gilberto", "Chong", "Rosendo", "Eddy", "Rob", "Dewitt", "Andre", "Titus", "Russell",
+            "Rigoberto", "Dick", "Garland", "Gabriel", "Hank", "Darius", "Ignacio", "Lazaro", "Johnie", "Mauro",
+            "Edmund", "Trent", "Harris", "Osvaldo", "Marvin", "Judson", "Rodney", "Randall", "Renato", "Richard",
+            "Denny", "Jon", "Doyle", "Cristopher", "Wilson", "Christian", "Jamie", "Roland", "Ken", "Tad", "Romeo",
+            "Seth", "Quinton", "Byron", "Ruben", "Darrel", "Deandre", "Broderick", "Harold", "Ty", "Monroe", "Landon",
+            "Mohammed", "Angel", "Arlen", "Elias", "Andres", "Carlton", "Numbers", "Tony", "Thaddeus", "Issac",
+            "Elmer", "Antoine", "Ned", "Fermin", "Grover", "Benito", "Abdul", "Cortez", "Eric", "Maxwell", "Coy",
+            "Gavin", "Rich", "Andy", "Del", "Giovanni", "Major", "Efren", "Horacio", "Joaquin", "Charles", "Noah",
+            "Deon", "Pasquale", "Reed", "Fausto", "Jermaine", "Irvin", "Ray", "Tobias", "Carter", "Yong", "Jorge",
+            "Brent", "Daniel", "Zane", "Walker", "Thad", "Shaun", "Jaime", "Mckinley", "Bradford", "Nathanial",
+            "Jerald", "Aubrey", "Virgil", "Abel", "Philip", "Chester", "Chadwick", "Dominick", "Britt", "Emmitt",
+            "Ferdinand", "Julian", "Reid", "Santos", "Dwain", "Morgan", "James", "Marion", "Micheal", "Eddie", "Brett",
+            "Stacy", "Kerry", "Dale", "Nicholas", "Darrick", "Freeman", "Scott", "Newton", "Sherman", "Felton",
+            "Cedrick", "Winfred", "Brad", "Fredric", "Dewayne", "Virgilio", "Reggie", "Edgar", "Heriberto", "Shad",
+            "Timmy", "Javier", "Nestor", "Royal", "Lynn", "Irwin", "Ismael", "Jonas", "Wiley", "Austin", "Kieth",
+            "Gonzalo", "Paris", "Earnest", "Arron", "Jarred", "Todd", "Erik", "Maria", "Chauncey", "Neil", "Conrad",
+            "Maurice", "Roosevelt", "Jacob", "Sydney", "Lee", "Basil", "Louis", "Rodolfo", "Rodger", "Roman", "Corey",
+            "Ambrose", "Cristobal", "Sylvester", "Benton", "Franklin", "Marcelo", "Guillermo", "Toby", "Jeramy",
+            "Donn", "Danny", "Dwight", "Clifford", "Valentine", "Matt", "Jules", "Kareem", "Ronny", "Lonny", "Son",
+            "Leopoldo", "Dannie", "Gregg", "Dillon", "Orlando", "Weston", "Kermit", "Damian", "Abraham", "Walton",
+            "Adrian", "Rudolf", "Will", "Les", "Norberto", "Fred", "Tyrone", "Ariel", "Terry", "Emmanuel", "Anderson",
+            "Elton", "Otis", "Derek", "Frankie", "Gino", "Lavern", "Jarod", "Kenny", "Dane", "Keenan", "Bryant",
+            "Eusebio", "Dorian", "Ali", "Lucas", "Wilford", "Jeremiah", "Warner", "Woodrow", "Galen", "Bob",
+            "Johnathon", "Amado", "Michel", "Harry", "Zachery", "Taylor", "Booker", "Hershel", "Mohammad", "Darrell",
+            "Kyle", "Stuart", "Marlin", "Hyman", "Jeffery", "Sidney", "Merrill", "Roy", "Garrett", "Porter", "Kenton",
+            "Giuseppe", "Terrance", "Trey", "Felix", "Buster", "Von", "Jackie", "Linwood", "Darron", "Francisco",
+            "Bernie", "Diego", "Brendon", "Cody", "Marco", "Ahmed", "Antonio", "Vince", "Brooks", "Kendrick", "Ross",
+            "Mohamed", "Jim", "Benny", "Gerald", "Pablo", "Charlie", "Antony", "Werner", "Hipolito", "Minh", "Mel",
+            "Derick", "Armand", "Fidel", "Lewis", "Donnell", "Desmond", "Vaughn", "Guadalupe", "Keneth", "Rodrick",
+            "Spencer", "Chas", "Gus", "Harlan", "Wes", "Carmelo", "Jefferson", "Gerard", "Jarvis", "Haywood", "Hayden",
+            "Sergio", "Gene", "Edgardo", "Colin", "Horace", "Dominic", "Aldo", "Adolfo", "Juan", "Man", "Lenard",
+            "Clement", "Everett", "Hal", "Bryon", "Mason", "Emerson", "Earle", "Laurence", "Columbus", "Lamar",
+            "Douglas", "Ian", "Fredrick", "Marc", "Loren", "Wallace", "Randell", "Noble", "Ricardo", "Rory", "Lindsey",
+            "Boris", "Bill", "Carlos", "Domingo", "Grant", "Craig", "Ezra", "Matthew", "Van", "Rudy", "Danial",
+            "Brock", "Maynard", "Vincent", "Cole", "Damion", "Ellsworth", "Marcel", "Markus", "Rueben", "Tanner",
+            "Reyes", "Hung", "Kennith", "Lindsay", "Howard", "Ralph", "Jed", "Monte", "Garfield", "Avery", "Bernardo",
+            "Malcolm", "Sterling", "Ezequiel", "Kristofer", "Luciano", "Casey", "Rosario", "Ellis", "Quintin",
+            "Trevor", "Miquel", "Jordan", "Arthur", "Carson", "Tyron", "Grady", "Walter", "Jonathon", "Ricky",
+            "Bennie", "Terrence", "Dion", "Dusty", "Roderick", "Isaac", "Rodrigo", "Harrison", "Zack", "Dee", "Devin",
+            "Rey", "Ulysses", "Clint", "Greg", "Dino", "Frances", "Wade", "Franklyn", "Jude", "Bradly", "Salvador",
+            "Rocky", "Weldon", "Lloyd", "Milford", "Clarence", "Alec", "Allan", "Bobbie", "Oswaldo", "Wilfred",
+            "Raleigh", "Shelby", "Willy", "Alphonso", "Arnoldo", "Robbie", "Truman", "Nicky", "Quinn", "Damien",
+            "Lacy", "Marcos", "Parker", "Burt", "Carroll", "Denver", "Buck", "Dong", "Normand", "Billie", "Edwin",
+            "Troy", "Arden", "Rusty", "Tommy", "Kenneth", "Leo", "Claud", "Joel", "Kendall", "Dante", "Milo", "Cruz",
+            "Lucien", "Ramon", "Jarrett", "Scottie", "Deshawn", "Ronnie", "Pete", "Alonzo", "Whitney", "Stefan",
+            "Sebastian", "Edmond", "Enrique", "Branden", "Leonard", "Loyd", "Olin", "Ron", "Rhett", "Frederic",
+            "Orval", "Tyrell", "Gail", "Eli", "Antonia", "Malcom", "Sandy", "Stacey", "Nickolas", "Hosea", "Santo",
+            "Oscar", "Fletcher", "Dave", "Patrick", "Dewey", "Bo", "Vito", "Blaine", "Randy", "Robin", "Winston",
+            "Sammy", "Edwardo", "Manuel", "Valentin", "Stanford", "Filiberto", "Buddy", "Zachariah", "Johnnie",
+            "Elbert", "Paul", "Isreal", "Jerrold", "Leif", "Owen", "Sung", "Junior", "Raphael", "Josef", "Donte",
+            "Allen", "Florencio", "Raymond", "Lauren", "Collin", "Eliseo", "Bruno", "Martin", "Lyndon", "Kurtis",
+            "Salvatore", "Erwin", "Michael", "Sean", "Davis", "Alberto", "King", "Rolland", "Joe", "Tory", "Chase",
+            "Dallas", "Vernon", "Beau", "Terrell", "Reynaldo", "Monty", "Jame", "Dirk", "Florentino", "Reuben", "Saul",
+            "Emory", "Esteban", "Michale", "Claudio", "Jacinto", "Kelley", "Levi", "Andrea", "Lanny", "Wendell",
+            "Elwood", "Joan", "Felipe", "Palmer", "Elmo", "Lawrence", "Hubert", "Rudolph", "Duane", "Cordell",
+            "Everette", "Mack", "Alan", "Efrain", "Trenton", "Bryan", "Tom", "Wilmer", "Clyde", "Chance", "Lou",
+            "Brain", "Justin", "Phil", "Jerrod", "George", "Kris", "Cyrus", "Emery", "Rickey", "Lincoln", "Renaldo",
+            "Mathew", "Luke", "Dwayne", "Alexis", "Jackson", "Gil", "Marty", "Burton", "Emil", "Glen", "Willian",
+            "Clemente", "Keven", "Barney", "Odell", "Reginald", "Aurelio", "Damon", "Ward", "Gustavo", "Harley",
+            "Peter", "Anibal", "Arlie", "Nigel", "Oren", "Zachary", "Scot", "Bud", "Wilbert", "Bart", "Josiah",
+            "Marlon", "Eldon", "Darryl", "Roger", "Anthony", "Omer", "Francis", "Patricia", "Moises", "Chuck",
+            "Waylon", "Hector", "Jamaal", "Cesar", "Julius", "Rex", "Norris", "Ollie", "Isaias", "Quentin", "Graig",
+            "Lyle", "Jeffry", "Karl", "Lester", "Danilo", "Mike", "Dylan", "Carlo", "Ryan", "Leon", "Percy", "Lucius",
+            "Jamel", "Lesley", "Joey", "Cornelius", "Rico", "Arnulfo", "Chet", "Margarito", "Ernie", "Nathanael",
+            "Amos", "Cleveland", "Luigi", "Alfonzo", "Phillip", "Clair", "Elroy", "Alva", "Hans", "Shon", "Gary",
+            "Jesus", "Cary", "Silas", "Keith", "Israel", "Willard", "Randolph", "Dan", "Adalberto", "Claude",
+            "Delbert", "Garry", "Mary", "Larry", "Riley", "Robt", "Darwin", "Barrett", "Steven", "Kelly", "Herschel",
+            "Darnell", "Scotty", "Armando", "Miguel", "Lawerence", "Wesley", "Garth", "Carol", "Micah", "Alvin",
+            "Billy", "Earl", "Pat", "Brady", "Cory", "Carey", "Bernard", "Jayson", "Nathaniel", "Gaylord", "Archie",
+            "Dorsey", "Erasmo", "Angelo", "Elisha", "Long", "Augustus", "Hobert", "Drew", "Stan", "Sherwood",
+            "Lorenzo", "Forrest", "Shawn", "Leigh", "Hiram", "Leonardo", "Gerry", "Myron", "Hugo", "Alvaro", "Leland",
+            "Genaro", "Jamey", "Stewart", "Elden", "Irving", "Olen", "Antone", "Freddy", "Lupe", "Joshua", "Gregory",
+            "Andrew", "Sang", "Wilbur", "Gerardo", "Merlin", "Williams", "Johnny", "Alex", "Tommie", "Jimmy",
+            "Donovan", "Dexter", "Gaston", "Tracy", "Jeff", "Stephen", "Berry", "Anton", "Darell", "Fritz", "Willis",
+            "Noel", "Mariano", "Crawford", "Zoey", "Alex", "Brianna", "Carlie", "Lloyd", "Cal", "Astor", "Randolf",
+            "Magdalene", "Trevelyan", "Terance", "Roy", "Kermit", "Harriett", "Crystal", "Laurinda", "Kiersten",
+            "Phyllida", "Liz", "Bettie", "Rena", "Colten", "Berenice", "Sindy", "Wilma", "Amos", "Candi", "Ritchie",
+            "Dirk", "Kathlyn", "Callista", "Anona", "Flossie", "Sterling", "Calista", "Regan", "Erica", "Jeana",
+            "Keaton", "York", "Nolan", "Daniel", "Benton", "Tommie", "Serenity", "Deanna", "Chas", "Heron", "Marlyn",
+            "Xylia", "Tristin", "Lyndon", "Andriana", "Madelaine", "Maddison", "Leila", "Chantelle", "Audrey",
+            "Connor", "Daley", "Tracee", "Tilda", "Eliot", "Merle", "Linwood", "Kathryn", "Silas", "Alvina",
+            "Phinehas", "Janis", "Alvena", "Zubin", "Gwendolen", "Caitlyn", "Bertram", "Hailee", "Idelle", "Homer",
+            "Jannah", "Delbert", "Rhianna", "Cy", "Jefferson", "Wayland", "Nona", "Tempest", "Reed", "Jenifer",
+            "Ellery", "Nicolina", "Aldous", "Prince", "Lexia", "Vinnie", "Doug", "Alberic", "Kayleen", "Woody",
+            "Rosanne", "Ysabel", "Skyler", "Twyla", "Geordie", "Leta", "Clive", "Aaron", "Scottie", "Celeste", "Chuck",
+            "Erle", "Lallie", "Jaycob", "Ray", "Carrie", "Laurita", "Noreen", "Meaghan", "Ulysses", "Andy", "Drogo",
+            "Dina", "Yasmin", "Mya", "Luvenia", "Urban", "Jacob", "Laetitia", "Sherry", "Love", "Michaela", "Deonne",
+            "Summer", "Brendon", "Sheena", "Mason", "Jayson", "Linden", "Salal", "Darrell", "Diana", "Hudson",
+            "Lennon", "Isador", "Charley", "April", "Ralph", "James", "Mina", "Jolyon", "Laurine", "Monna", "Carita",
+            "Munro", "Elsdon", "Everette", "Radclyffe", "Darrin", "Herbert", "Gawain", "Sheree", "Trudy", "Emmaline",
+            "Kassandra", "Rebecca", "Basil", "Jen", "Don", "Osborne", "Lilith", "Hannah", "Fox", "Rupert", "Paulene",
+            "Darius", "Wally", "Baptist", "Sapphire", "Tia", "Sondra", "Kylee", "Ashton", "Jepson", "Joetta", "Val",
+            "Adela", "Zacharias", "Zola", "Marmaduke", "Shannah", "Posie", "Oralie", "Brittany", "Ernesta", "Raymund",
+            "Denzil", "Daren", "Roosevelt", "Nelson", "Fortune", "Mariel", "Nick", "Jaden", "Upton", "Oz", "Margaux",
+            "Precious", "Albert", "Bridger", "Jimmy", "Nicola", "Rosalynne", "Keith", "Walt", "Della", "Joanna",
+            "Xenia", "Esmeralda", "Major", "Simon", "Rexana", "Stacy", "Calanthe", "Sherley", "Kaitlyn", "Graham",
+            "Ramsey", "Abbey", "Madlyn", "Kelvin", "Bill", "Rue", "Monica", "Caileigh", "Laraine", "Booker", "Jayna",
+            "Greta", "Jervis", "Sherman", "Kendrick", "Tommy", "Iris", "Geffrey", "Kaelea", "Kerr", "Garrick", "Jep",
+            "Audley", "Nic", "Bronte", "Beulah", "Patricia", "Jewell", "Deidra", "Cory", "Everett", "Harper",
+            "Charity", "Godfrey", "Jaime", "Sinclair", "Talbot", "Dayna", "Cooper", "Rosaline", "Jennie", "Eileen",
+            "Latanya", "Corinna", "Roxie", "Caesar", "Charles", "Pollie", "Lindsey", "Sorrel", "Dwight", "Jocelyn",
+            "Weston", "Shyla", "Valorie", "Bessie", "Josh", "Lessie", "Dayton", "Kathi", "Chasity", "Wilton", "Adam",
+            "William", "Ash", "Angela", "Ivor", "Ria", "Jazmine", "Hailey", "Jo", "Silvestra", "Ernie", "Clifford",
+            "Levi", "Matilda", "Quincey", "Camilla", "Delicia", "Phemie", "Laurena", "Bambi", "Lourdes", "Royston",
+            "Chastity", "Lynwood", "Elle", "Brenda", "Phoebe", "Timothy", "Raschelle", "Lilly", "Burt", "Rina",
+            "Rodney", "Maris", "Jaron", "Wilf", "Harlan", "Audra", "Vincent", "Elwyn", "Drew", "Wynter", "Ora",
+            "Lissa", "Virgil", "Xavier", "Chad", "Ollie", "Leyton", "Karolyn", "Skye", "Roni", "Gladys", "Dinah",
+            "Penny", "August", "Osmund", "Whitaker", "Brande", "Cornell", "Phil", "Zara", "Kilie", "Gavin", "Coty",
+            "Randy", "Teri", "Keira", "Pru", "Clemency", "Kelcey", "Nevil", "Poppy", "Gareth", "Christabel", "Bastian",
+            "Wynonna", "Roselyn", "Goddard", "Collin", "Trace", "Neal", "Effie", "Denys", "Virginia", "Richard",
+            "Isiah", "Harrietta", "Gaylord", "Diamond", "Trudi", "Elaine", "Jemmy", "Gage", "Annabel", "Quincy", "Syd",
+            "Marianna", "Philomena", "Aubree", "Kathie", "Jacki", "Kelley", "Bess", "Cecil", "Maryvonne", "Kassidy",
+            "Anselm", "Dona", "Darby", "Jamison", "Daryl", "Darell", "Teal", "Lennie", "Bartholomew", "Katie",
+            "Maybelline", "Kimball", "Elvis", "Les", "Flick", "Harley", "Beth", "Bidelia", "Montague", "Helen", "Ozzy",
+            "Stef", "Debra", "Maxene", "Stefanie", "Russ", "Avril", "Johnathan", "Orson", "Chelsey", "Josephine",
+            "Deshaun", "Wendell", "Lula", "Ferdinanda", "Greg", "Brad", "Kynaston", "Dena", "Russel", "Robertina",
+            "Misti", "Leon", "Anjelica", "Bryana", "Myles", "Judi", "Curtis", "Davin", "Kristia", "Chrysanta",
+            "Hayleigh", "Hector", "Osbert", "Eustace", "Cary", "Tansy", "Cayley", "Maryann", "Alissa", "Ike",
+            "Tranter", "Reina", "Alwilda", "Sidony", "Columbine", "Astra", "Jillie", "Stephania", "Jonah", "Kennedy",
+            "Ferdinand", "Allegria", "Donella", "Kelleigh", "Darian", "Eldreda", "Jayden", "Herbie", "Jake", "Winston",
+            "Vi", "Annie", "Cherice", "Hugo", "Tricia", "Haydee", "Cassarah", "Darden", "Mallory", "Alton", "Hadley",
+            "Romayne", "Lacey", "Ern", "Alayna", "Cecilia", "Seward", "Tilly", "Edgar", "Concordia", "Ibbie", "Dahlia",
+            "Oswin", "Stu", "Brett", "Maralyn", "Kristeen", "Dotty", "Robyn", "Nessa", "Tresha", "Guinevere",
+            "Emerson", "Haze", "Lyn", "Henderson", "Lexa", "Jaylen", "Gail", "Lizette", "Tiara", "Robbie", "Destiny",
+            "Alice", "Livia", "Rosy", "Leah", "Jan", "Zach", "Vita", "Gia", "Micheal", "Rowina", "Alysha", "Bobbi",
+            "Delores", "Osmond", "Karaugh", "Wilbur", "Kasandra", "Renae", "Kaety", "Dora", "Gaye", "Amaryllis",
+            "Katelyn", "Dacre", "Prudence", "Ebony", "Camron", "Jerrold", "Vivyan", "Randall", "Donna", "Misty",
+            "Damon", "Selby", "Esmund", "Rian", "Garry", "Julius", "Raelene", "Clement", "Dom", "Tibby", "Moss",
+            "Millicent", "Gwendoline", "Berry", "Ashleigh", "Lilac", "Quin", "Vere", "Creighton", "Harriet", "Malvina",
+            "Lianne", "Pearle", "Kizzie", "Kara", "Petula", "Jeanie", "Maria", "Pacey", "Victoria", "Huey", "Toni",
+            "Rose", "Wallis", "Diggory", "Josiah", "Delma", "Keysha", "Channing", "Prue", "Lee", "Ryan", "Sidney",
+            "Valerie", "Clancy", "Ezra", "Gilbert", "Clare", "Laz", "Crofton", "Mike", "Annabella", "Tara", "Eldred",
+            "Arthur", "Jaylon", "Peronel", "Paden", "Dot", "Marian", "Amyas", "Alexus", "Esmond", "Abbie", "Stanley",
+            "Brittani", "Vickie", "Errol", "Kimberlee", "Uland", "Ebenezer", "Howie", "Eveline", "Andrea", "Trish",
+            "Hopkin", "Bryanna", "Temperance", "Valarie", "Femie", "Alix", "Terrell", "Lewin", "Lorrin", "Happy",
+            "Micah", "Rachyl", "Sloan", "Gertrude", "Elizabeth", "Dorris", "Andra", "Bram", "Gary", "Jeannine",
+            "Maurene", "Irene", "Yolonda", "Jonty", "Coleen", "Cecelia", "Chantal", "Stuart", "Caris", "Ros",
+            "Kaleigh", "Mirabelle", "Kolby", "Primrose", "Susannah", "Ginny", "Jinny", "Dolly", "Lettice", "Sonny",
+            "Melva", "Ernest", "Garret", "Reagan", "Trenton", "Gallagher", "Edwin", "Nikolas", "Corrie", "Lynette",
+            "Ettie", "Sly", "Debbi", "Eudora", "Brittney", "Tacey", "Marius", "Anima", "Gordon", "Olivia", "Kortney",
+            "Shantel", "Kolleen", "Nevaeh", "Buck", "Sera", "Liliana", "Aric", "Kalyn", "Mick", "Libby", "Ingram",
+            "Alexandria", "Darleen", "Jacklyn", "Hughie", "Tyler", "Aida", "Ronda", "Deemer", "Taryn", "Laureen",
+            "Samantha", "Dave", "Hardy", "Baldric", "Montgomery", "Gus", "Ellis", "Titania", "Luke", "Chase", "Haidee",
+            "Mayra", "Isabell", "Trinity", "Milo", "Abigail", "Tacita", "Meg", "Hervey", "Natasha", "Sadie", "Holden",
+            "Dee", "Mansel", "Perry", "Randi", "Frederica", "Georgina", "Kolour", "Debbie", "Seraphina", "Elspet",
+            "Julyan", "Raven", "Zavia", "Jarvis", "Jaymes", "Grover", "Cairo", "Alea", "Jordon", "Braxton", "Donny",
+            "Rhoda", "Tonya", "Bee", "Alyssia", "Ashlyn", "Reanna", "Lonny", "Arlene", "Deb", "Jane", "Nikole",
+            "Bettina", "Harrison", "Tamzen", "Arielle", "Adelaide", "Faith", "Bridie", "Wilburn", "Fern", "Nan",
+            "Shaw", "Zeke", "Alan", "Dene", "Gina", "Alexa", "Bailey", "Sal", "Tammy", "Maximillian", "America",
+            "Sylvana", "Fitz", "Mo", "Marissa", "Cass", "Eldon", "Wilfrid", "Tel", "Joann", "Kendra", "Tolly",
+            "Leanne", "Ferdie", "Haven", "Lucas", "Marlee", "Cyrilla", "Red", "Phoenix", "Jazmin", "Carin", "Gena",
+            "Lashonda", "Tucker", "Genette", "Kizzy", "Winifred", "Melody", "Keely", "Kaylyn", "Radcliff", "Lettie",
+            "Foster", "Lyndsey", "Nicholas", "Farley", "Louisa", "Dana", "Dortha", "Francine", "Doran", "Bonita",
+            "Hal", "Sawyer", "Reginald", "Aislin", "Nathan", "Baylee", "Abilene", "Ladonna", "Maurine", "Shelly",
+            "Deandre", "Jasmin", "Roderic", "Tiffany", "Amanda", "Verity", "Wilford", "Gayelord", "Whitney", "Demelza",
+            "Kenton", "Alberta", "Kyra", "Tabitha", "Sampson", "Korey", "Lillian", "Edison", "Clayton", "Steph",
+            "Maya", "Dusty", "Jim", "Ronny", "Adrianne", "Bernard", "Harris", "Kiley", "Alexander", "Kisha", "Ethalyn",
+            "Patience", "Briony", "Indigo", "Aureole", "Makenzie", "Molly", "Sherilyn", "Barry", "Laverne", "Hunter",
+            "Rocky", "Tyreek", "Madalyn", "Phyliss", "Chet", "Beatrice", "Faye", "Lavina", "Madelyn", "Tracey",
+            "Gyles", "Patti", "Carlyn", "Stephanie", "Jackalyn", "Larrie", "Kimmy", "Isolda", "Emelina", "Lis",
+            "Zillah", "Cody", "Sheard", "Rufus", "Paget", "Mae", "Rexanne", "Luvinia", "Tamsen", "Rosanna", "Greig",
+            "Stacia", "Mabelle", "Quianna", "Lotus", "Delice", "Bradford", "Angus", "Cosmo", "Earlene", "Adrian",
+            "Arlie", "Noelle", "Sabella", "Isa", "Adelle", "Innocent", "Kirby", "Trixie", "Kenelm", "Nelda", "Melia",
+            "Kendal", "Dorinda", "Placid", "Linette", "Kam", "Sherisse", "Evan", "Ewart", "Janice", "Linton",
+            "Jacaline", "Charissa", "Douglas", "Aileen", "Kemp", "Oli", "Amethyst", "Rosie", "Nigella", "Sherill",
+            "Anderson", "Alanna", "Eric", "Claudia", "Jennifer", "Boniface", "Harriet", "Vernon", "Lucy", "Shawnee",
+            "Gerard", "Cecily", "Romey", "Randall", "Wade", "Lux", "Dawson", "Gregg", "Kade", "Roxanne", "Melinda",
+            "Rolland", "Rowanne", "Fannie", "Isidore", "Melia", "Harvie", "Salal", "Eleonor", "Jacquette", "Lavone",
+            "Shanika", "Tarquin", "Janet", "Josslyn", "Maegan", "Augusta", "Aubree", "Francene", "Martie", "Marisa",
+            "Tyreek", "Tatianna", "Caleb", "Sheridan", "Nellie", "Barbara", "Wat", "Jayla", "Esmaralda", "Graeme",
+            "Lavena", "Jemima", "Nikolas", "Triston", "Portia", "Kyla", "Marcus", "Raeburn", "Jamison", "Earl", "Wren",
+            "Leighton", "Lagina", "Lucasta", "Dina", "Amaranta", "Jessika", "Claud", "Bernard", "Winifred", "Ebba",
+            "Sammi", "Gall", "Chloe", "Ottoline", "Herbert", "Janice", "Gareth", "Channing", "Caleigh", "Kailee",
+            "Ralphie", "Tamzen", "Quincy", "Beaumont", "Albert", "Jadyn", "Violet", "Luanna", "Moriah", "Humbert",
+            "Jed", "Leona", "Hale", "Mitch", "Marlin", "Nivek", "Darwin", "Dirk", "Liliana", "Meadow", "Bernadine",
+            "Jorie", "Peyton", "Astra", "Roscoe", "Gina", "Lovell", "Jewel", "Romayne", "Rosy", "Imogene",
+            "Margaretta", "Lorinda", "Hopkin", "Bobby", "Flossie", "Bennie", "Horatio", "Jonah", "Lyn", "Deana",
+            "Juliana", "Blanch", "Wright", "Kendal", "Woodrow", "Tania", "Austyn", "Val", "Mona", "Charla", "Rudyard",
+            "Pamela", "Raven", "Zena", "Nicola", "Kaelea", "Conor", "Virgil", "Sonnie", "Goodwin", "Christianne",
+            "Linford", "Myron", "Denton", "Charita", "Brody", "Ginnie", "Harrison", "Jeanine", "Quin", "Isolda",
+            "Zoie", "Pearce", "Margie", "Larrie", "Angelina", "Marcia", "Jessamine", "Delilah", "Dick", "Luana",
+            "Delicia", "Lake", "Luvenia", "Vaughan", "Concordia", "Gayelord", "Cheyenne", "Felix", "Dorris", "Pen",
+            "Kristeen", "Parris", "Everitt", "Josephina", "Amy", "Tommie", "Adrian", "April", "Rosaline", "Zachery",
+            "Trace", "Phoebe", "Jenelle", "Kameron", "Katharine", "Media", "Colton", "Tad", "Quianna", "Kerenza",
+            "Greta", "Luvinia", "Pete", "Tonya", "Beckah", "Barbra", "Jon", "Tetty", "Corey", "Sylvana", "Kizzy",
+            "Korey", "Trey", "Haydee", "Penny", "Mandy", "Panda", "Coline", "Ramsey", "Sukie", "Annabel", "Sarina",
+            "Corbin", "Suzanna", "Rob", "Duana", "Shell", "Jason", "Eddy", "Rube", "Roseann", "Celia", "Brianne",
+            "Nerissa", "Jera", "Humphry", "Ashlynn", "Terrence", "Philippina", "Coreen", "Kolour", "Indiana", "Paget",
+            "Marlyn", "Hester", "Isbel", "Ocean", "Harris", "Leslie", "Vere", "Monroe", "Isabelle", "Bertie", "Clitus",
+            "Dave", "Alethea", "Lessie", "Louiza", "Madlyn", "Garland", "Wolf", "Lalo", "Donny", "Amabel", "Tianna",
+            "Louie", "Susie", "Mackenzie", "Renie", "Tess", "Marmaduke", "Gwendolen", "Bettina", "Beatrix", "Esmund",
+            "Minnie", "Carlie", "Barnabas", "Ruthie", "Honour", "Haylie", "Xavior", "Freddie", "Ericka", "Aretha",
+            "Edie", "Madelina", "Anson", "Tabby", "Derrick", "Jocosa", "Deirdre", "Aislin", "Chastity", "Abigail",
+            "Wynonna", "Zo", "Eldon", "Krystine", "Ghislaine", "Zavia", "Nolene", "Marigold", "Kelley", "Sylvester",
+            "Odell", "George", "Laurene", "Franklyn", "Clarice", "Mo", "Dustin", "Debbi", "Lina", "Tony", "Acacia",
+            "Hettie", "Natalee", "Marcie", "Brittany", "Elnora", "Rachel", "Dawn", "Basil", "Christal", "Anjelica",
+            "Fran", "Tawny", "Delroy", "Tameka", "Lillie", "Ceara", "Deanna", "Deshaun", "Ken", "Bradford", "Justina",
+            "Merle", "Draven", "Gretta", "Harriette", "Webster", "Nathaniel", "Anemone", "Coleen", "Ruth", "Chryssa",
+            "Hortensia", "Saffie", "Deonne", "Leopold", "Harlan", "Lea", "Eppie", "Lucinda", "Tilda", "Fanny", "Titty",
+            "Lockie", "Jepson", "Sherisse", "Maralyn", "Ethel", "Sly", "Ebenezer", "Canute", "Ella", "Freeman",
+            "Reuben", "Olivette", "Nona", "Rik", "Amice", "Kristine", "Kathie", "Jayne", "Jeri", "Mckenna", "Bertram",
+            "Kaylee", "Livia", "Gil", "Wallace", "Maryann", "Keeleigh", "Laurinda", "Doran", "Khloe", "Dakota",
+            "Yaron", "Kimberleigh", "Gytha", "Doris", "Marylyn", "Benton", "Linnette", "Esther", "Jakki", "Rowina",
+            "Marian", "Roselyn", "Norbert", "Maggie", "Caesar", "Phinehas", "Jerry", "Jasmine", "Antonette", "Miriam",
+            "Monna", "Maryvonne", "Jacquetta", "Bernetta", "Napier", "Annie", "Gladwin", "Sheldon", "Aric", "Elouise",
+            "Gawain", "Kristia", "Gabe", "Kyra", "Red", "Tod", "Dudley", "Lorraine", "Ryley", "Sabina", "Poppy",
+            "Leland", "Aileen", "Eglantine", "Alicia", "Jeni", "Addy", "Tiffany", "Geffrey", "Lavina", "Collin",
+            "Clover", "Vin", "Jerome", "Doug", "Vincent", "Florence", "Scarlet", "Celeste", "Desdemona", "Tiphanie",
+            "Kassandra", "Ashton", "Madison", "Art", "Magdalene", "Iona", "Josepha", "Anise", "Ferne", "Derek",
+            "Huffie", "Qiana", "Ysabel", "Tami", "Shannah", "Xavier", "Willard", "Winthrop", "Vickie", "Maura",
+            "Placid", "Tiara", "Reggie", "Elissa", "Isa", "Chrysanta", "Jeff", "Bessie", "Terri", "Amilia", "Brett",
+            "Daniella", "Damion", "Carolina", "Maximillian", "Travers", "Benjamin", "Oprah", "Darcy", "Yolanda",
+            "Nicolina", "Crofton", "Jarrett", "Kaitlin", "Shauna", "Keren", "Bevis", "Kalysta", "Sharron", "Alyssa",
+            "Blythe", "Zelma", "Caelie", "Norwood", "Billie", "Patrick", "Gary", "Cambria", "Tylar", "Mason", "Helen",
+            "Melyssa", "Gene", "Gilberta", "Carter", "Herbie", "Harmonie", "Leola", "Eugenia", "Clint", "Pauletta",
+            "Edwyna", "Georgina", "Teal", "Harper", "Izzy", "Dillon", "Kezia", "Evangeline", "Colene", "Madelaine",
+            "Zilla", "Rudy", "Dottie", "Caris", "Morton", "Marge", "Tacey", "Parker", "Troy", "Liza", "Lewin",
+            "Tracie", "Justine", "Dallas", "Linden", "Ray", "Loretta", "Teri", "Elvis", "Diane", "Julianna", "Manfred",
+            "Denise", "Eireen", "Ann", "Kenith", "Linwood", "Kathlyn", "Bernice", "Shelley", "Oswald", "Amedeus",
+            "Homer", "Tanzi", "Ted", "Ralphina", "Hyacinth", "Lotus", "Matthias", "Arlette", "Clark", "Cecil",
+            "Elspeth", "Alvena", "Noah", "Millard", "Brenden", "Cole", "Philipa", "Nina", "Thelma", "Iantha", "Reid",
+            "Jefferson", "Meg", "Elsie", "Shirlee", "Nathan", "Nancy", "Simona", "Racheal", "Carin", "Emory", "Delice",
+            "Kristi", "Karaugh", "Kaety", "Tilly", "Em", "Alanis", "Darrin", "Jerrie", "Hollis", "Cary", "Marly",
+            "Carita", "Jody", "Farley", "Hervey", "Rosalin", "Cuthbert", "Stewart", "Jodene", "Caileigh", "Briscoe",
+            "Dolores", "Sheree", "Eustace", "Nigel", "Detta", "Barret", "Rowland", "Kenny", "Githa", "Zoey", "Adela",
+            "Petronella", "Opal", "Coleman", "Niles", "Cyril", "Dona", "Alberic", "Allannah", "Jules", "Avalon",
+            "Hadley", "Thomas", "Renita", "Calanthe", "Heron", "Shawnda", "Chet", "Malina", "Manny", "Rina", "Frieda",
+            "Eveleen", "Deshawn", "Amos", "Raelene", "Paige", "Molly", "Nannie", "Ileen", "Brendon", "Milford",
+            "Unice", "Rebeccah", "Caedmon", "Gae", "Doreen", "Vivian", "Louis", "Raphael", "Vergil", "Lise", "Glenn",
+            "Karyn", "Terance", "Reina", "Jake", "Gordon", "Wisdom", "Isiah", "Gervase", "Fern", "Marylou", "Roddy",
+            "Justy", "Derick", "Shantelle", "Adam", "Chantel", "Madoline", "Emmerson", "Lexie", "Mickey", "Stephen",
+            "Dane", "Stacee", "Elwin", "Tracey", "Alexandra", "Ricky", "Ian", "Kasey", "Rita", "Alanna", "Georgene",
+            "Deon", "Zavier", "Ophelia", "Deforest", "Lowell", "Zubin", "Hardy", "Osmund", "Tabatha", "Debby",
+            "Katlyn", "Tallulah", "Priscilla", "Braden", "Wil", "Keziah", "Jen", "Aggie", "Korbin", "Lemoine",
+            "Barnaby", "Tranter", "Goldie", "Roderick", "Trina", "Emery", "Pris", "Sidony", "Adelle", "Tate", "Wilf",
+            "Zola", "Brande", "Chris", "Calanthia", "Lilly", "Kaycee", "Lashonda", "Jasmin", "Elijah", "Shantel",
+            "Simon", "Rosalind", "Jarod", "Kaylie", "Corrine", "Joselyn", "Archibald", "Mariabella", "Winton",
+            "Merlin", "Chad", "Ursula", "Kristopher", "Hewie", "Adrianna", "Lyndsay", "Jasmyn", "Tim", "Evette",
+            "Margaret", "Samson", "Bronte", "Terence", "Leila", "Candice", "Tori", "Jamey", "Coriander", "Conrad",
+            "Floyd", "Karen", "Lorin", "Maximilian", "Cairo", "Emily", "Yasmin", "Karolyn", "Bryan", "Lanny",
+            "Kimberly", "Rick", "Chaz", "Krystle", "Lyric", "Laura", "Garrick", "Flip", "Monty", "Brendan",
+            "Ermintrude", "Rayner", "Merla", "Titus", "Marva", "Patricia", "Leone", "Tracy", "Jaqueline", "Hallam",
+            "Delores", "Cressida", "Carlyle", "Leann", "Kelcey", "Laurence", "Ryan", "Reynold", "Mark", "Collyn",
+            "Audie", "Sammy", "Ellery", "Sallie", "Pamelia", "Adolph", "Lydia", "Titania", "Ron", "Bridger", "Aline",
+            "Read", "Kelleigh", "Weldon", "Irving", "Garey", "Diggory", "Evander", "Kylee", "Deidre", "Ormond",
+            "Laurine", "Reannon", "Arline", "Pat"
+
+    };
+
+    public static String[] jargon = { "wireless", "signal", "network", "3G", "plan", "touch-screen",
+            "customer-service", "reachability", "voice-command", "shortcut-menu", "customization", "platform", "speed",
+            "voice-clarity", "voicemail-service" };
+
+    public static String[] vendors = { "at&t", "verizon", "t-mobile", "sprint", "motorola", "samsung", "iphone" };
+
+    public static String[] org_list = { "Latsonity", "ganjalax", "Zuncan", "Lexitechno", "Hot-tech", "subtam",
+            "Coneflex", "Ganjatax", "physcane", "Tranzap", "Qvohouse", "Zununoing", "jaydax", "Keytech", "goldendexon",
+            "Villa-tech", "Trustbam", "Newcom", "Voltlane", "Ontohothex", "Ranhotfan", "Alphadax", "Transhigh",
+            "kin-ron", "Doublezone", "Solophase", "Vivaace", "silfind", "Basecone", "sonstreet", "Freshfix",
+            "Techitechi", "Kanelectrics", "linedexon", "Goldcity", "Newfase", "Technohow", "Zimcone", "Salthex",
+            "U-ron", "Solfix", "whitestreet", "Xx-technology", "Hexviafind", "over-it", "Strongtone", "Tripplelane",
+            "geomedia", "Scotcity", "Inchex", "Vaiatech", "Striptaxon", "Hatcom", "tresline", "Sanjodax", "freshdox",
+            "Sumlane", "Quadlane", "Newphase", "overtech", "Voltbam", "Icerunin", "Fixdintex", "Hexsanhex", "Statcode",
+            "Greencare", "U-electrics", "Zamcorporation", "Ontotanin", "Tanzimcare", "Groovetex", "Ganjastrip",
+            "Redelectronics", "Dandamace", "Whitemedia", "strongex", "Streettax", "highfax", "Mathtech", "Xx-drill",
+            "Sublamdox", "Unijobam", "Rungozoom", "Fixelectrics", "Villa-dox", "Ransaofan", "Plexlane", "itlab",
+            "Lexicone", "Fax-fax", "Viatechi", "Inchdox", "Kongreen", "Doncare", "Y-geohex", "Opeelectronics",
+            "Medflex", "Dancode", "Roundhex", "Labzatron", "Newhotplus", "Sancone", "Ronholdings", "Quoline",
+            "zoomplus", "Fix-touch", "Codetechno", "Tanzumbam", "Indiex", "Canline" };
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java
new file mode 100644
index 0000000..ad7daaf
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java
@@ -0,0 +1,2483 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class DataGenerator2 {
+
+    private RandomDateGenerator randDateGen;
+    private RandomNameGenerator randNameGen;
+    private RandomEmploymentGenerator randEmpGen;
+    private RandomMessageGenerator randMessageGen;
+    private RandomLocationGenerator randLocationGen;
+
+    private DistributionHandler fbDistHandler;
+    private DistributionHandler twDistHandler;
+
+    private int totalFbMessages;
+    private int numFbOnlyUsers;
+    private int totalTwMessages;
+    private int numTwOnlyUsers;
+
+    private int numCommonUsers;
+
+    private int fbUserId;
+    private int twUserId;
+
+    private int fbMessageId;
+    private int twMessageId;
+
+    private Random random = new Random();
+
+    private String commonUserFbSuffix = "_fb";
+    private String commonUserTwSuffix = "_tw";
+
+    private String outputDir;
+
+    private PartitionConfiguration partition;
+
+    private FacebookUser fbUser = new FacebookUser();
+    private TwitterUser twUser = new TwitterUser();
+
+    private FacebookMessage fbMessage = new FacebookMessage();
+    private TweetMessage twMessage = new TweetMessage();
+
+    private int duration;
+
+    private DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+
+    public DataGenerator2(String[] args) throws Exception {
+        String controllerInstallDir = args[0];
+        String partitionConfXML = controllerInstallDir + "/output/partition-conf.xml";
+        String partitionName = args[1];
+        partition = XMLUtil.getPartitionConfiguration(partitionConfXML, partitionName);
+
+        // 1
+        randDateGen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+
+        String firstNameFile = controllerInstallDir + "/metadata/firstNames.txt";
+        String lastNameFile = controllerInstallDir + "/metadata/lastNames.txt";
+        String vendorFile = controllerInstallDir + "/metadata/vendors.txt";
+        String jargonFile = controllerInstallDir + "/metadata/jargon.txt";
+        String orgList = controllerInstallDir + "/metadata/org_list.txt";
+
+        randNameGen = new RandomNameGenerator(firstNameFile, lastNameFile);
+        randEmpGen = new RandomEmploymentGenerator(90, new Date(1, 1, 2000), new Date(8, 20, 2012), orgList);
+        randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+        randMessageGen = new RandomMessageGenerator(vendorFile, jargonFile);
+
+        totalFbMessages = partition.getTargetPartition().getFbMessageIdMax()
+                - partition.getTargetPartition().getFbMessageIdMin() + 1;
+        numFbOnlyUsers = (partition.getTargetPartition().getFbUserKeyMax()
+                - partition.getTargetPartition().getFbUserKeyMin() + 1)
+                - partition.getTargetPartition().getCommonUsers();
+
+        totalTwMessages = partition.getTargetPartition().getTwMessageIdMax()
+                - partition.getTargetPartition().getTwMessageIdMin() + 1;
+        numTwOnlyUsers = (partition.getTargetPartition().getTwUserKeyMax()
+                - partition.getTargetPartition().getTwUserKeyMin() + 1)
+                - partition.getTargetPartition().getCommonUsers();
+
+        numCommonUsers = partition.getTargetPartition().getCommonUsers();
+        fbDistHandler = new DistributionHandler(totalFbMessages, 0.5, numFbOnlyUsers + numCommonUsers);
+        twDistHandler = new DistributionHandler(totalTwMessages, 0.5, numTwOnlyUsers + numCommonUsers);
+
+        fbUserId = partition.getTargetPartition().getFbUserKeyMin();
+        twUserId = partition.getTargetPartition().getTwUserKeyMin();
+
+        fbMessageId = partition.getTargetPartition().getFbMessageIdMin();
+        twMessageId = partition.getTargetPartition().getTwMessageIdMin();
+
+        outputDir = partition.getSourcePartition().getPath();
+    }
+
+    public DataGenerator2(InitializationInfo info) {
+        initialize(info);
+    }
+
+    private void generateFacebookOnlyUsers(int numFacebookUsers) throws IOException {
+        FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, true);
+        FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, true);
+
+        for (int i = 0; i < numFacebookUsers; i++) {
+            getFacebookUser(null);
+            appender.appendToFile(fbUser.toString());
+            generateFacebookMessages(fbUser, messageAppender, -1);
+        }
+        appender.close();
+        messageAppender.close();
+    }
+
+    private void generateTwitterOnlyUsers(int numTwitterUsers) throws IOException {
+        FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, true);
+        FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, true);
+
+        for (int i = 0; i < numTwitterUsers; i++) {
+            getTwitterUser(null);
+            appender.appendToFile(twUser.toString());
+            generateTwitterMessages(twUser, messageAppender, -1);
+        }
+        appender.close();
+        messageAppender.close();
+    }
+
+    private void generateCommonUsers() throws IOException {
+        FileAppender fbAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, false);
+        FileAppender twAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, false);
+        FileAppender fbMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, false);
+        FileAppender twMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, false);
+
+        for (int i = 0; i < numCommonUsers; i++) {
+            getFacebookUser(commonUserFbSuffix);
+            fbAppender.appendToFile(fbUser.toString());
+            generateFacebookMessages(fbUser, fbMessageAppender, -1);
+
+            getCorrespondingTwitterUser(fbUser);
+            twAppender.appendToFile(twUser.toString());
+            generateTwitterMessages(twUser, twMessageAppender, -1);
+        }
+
+        fbAppender.close();
+        twAppender.close();
+        fbMessageAppender.close();
+        twMessageAppender.close();
+    }
+
+    private void generateFacebookMessages(FacebookUser user, FileAppender appender, int numMsg) throws IOException {
+        Message message;
+        int numMessages = 0;
+        if (numMsg == -1) {
+            numMessages = fbDistHandler
+                    .getFromDistribution(fbUserId - partition.getTargetPartition().getFbUserKeyMin());
+        }
+        for (int i = 0; i < numMessages; i++) {
+            message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            fbMessage.reset(fbMessageId++, user.getId(), random.nextInt(totalFbMessages + 1), location, message);
+            appender.appendToFile(fbMessage.toString());
+        }
+    }
+
+    private void generateTwitterMessages(TwitterUser user, FileAppender appender, int numMsg) throws IOException {
+        Message message;
+        int numMessages = 0;
+        if (numMsg == -1) {
+            numMessages = twDistHandler
+                    .getFromDistribution(twUserId - partition.getTargetPartition().getTwUserKeyMin());
+        }
+
+        for (int i = 0; i < numMessages; i++) {
+            message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            DateTime sendTime = randDateGen.getNextRandomDatetime();
+            twMessage.reset(twMessageId + "", user, location, sendTime, message.getReferredTopics(), message);
+            twMessageId++;
+            appender.appendToFile(twMessage.toString());
+        }
+    }
+
+    public Iterator<TweetMessage> getTwitterMessageIterator() {
+        return new TweetMessageIterator(duration);
+    }
+
+    public class TweetMessageIterator implements Iterator<TweetMessage> {
+
+        private final int duration;
+        private long startTime = 0;
+
+        public TweetMessageIterator(int duration) {
+            this.duration = duration;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (startTime == 0) {
+                startTime = System.currentTimeMillis();
+            }
+            return System.currentTimeMillis() - startTime < duration * 1000;
+        }
+
+        @Override
+        public TweetMessage next() {
+            getTwitterUser(null);
+            Message message = randMessageGen.getNextRandomMessage();
+            Point location = randLocationGen.getRandomPoint();
+            DateTime sendTime = randDateGen.getNextRandomDatetime();
+            twMessage.reset(UUID.randomUUID().toString(), twUser, location, sendTime, message.getReferredTopics(),
+                    message);
+            twMessageId++;
+            if (twUserId > numTwOnlyUsers) {
+                twUserId = 1;
+            }
+            return twMessage;
+
+        }
+
+        @Override
+        public void remove() {
+            // TODO Auto-generated method stub
+
+        }
+
+    }
+
+    public static class InitializationInfo {
+        public Date startDate = new Date(1, 1, 2005);
+        public Date endDate = new Date(8, 20, 2012);
+        public String[] lastNames = DataGenerator.lastNames;
+        public String[] firstNames = DataGenerator.firstNames;
+        public String[] vendors = DataGenerator.vendors;
+        public String[] jargon = DataGenerator.jargon;
+        public String[] org_list = DataGenerator.org_list;
+        public int percentEmployed = 90;
+        public Date employmentStartDate = new Date(1, 1, 2000);
+        public Date employmentEndDate = new Date(31, 12, 2012);
+        public int totalFbMessages;
+        public int numFbOnlyUsers;
+        public int totalTwMessages;
+        public int numTwOnlyUsers = 5000;
+        public int numCommonUsers;
+        public int fbUserIdMin;
+        public int fbMessageIdMin;
+        public int twUserIdMin;
+        public int twMessageIdMin;
+        public int timeDurationInSecs = 60;
+
+    }
+
+    public void initialize(InitializationInfo info) {
+        randDateGen = new RandomDateGenerator(info.startDate, info.endDate);
+        randNameGen = new RandomNameGenerator(info.firstNames, info.lastNames);
+        randEmpGen = new RandomEmploymentGenerator(info.percentEmployed, info.employmentStartDate,
+                info.employmentEndDate, info.org_list);
+        randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+        randMessageGen = new RandomMessageGenerator(info.vendors, info.jargon);
+        fbDistHandler = new DistributionHandler(info.totalFbMessages, 0.5, info.numFbOnlyUsers + info.numCommonUsers);
+        twDistHandler = new DistributionHandler(info.totalTwMessages, 0.5, info.numTwOnlyUsers + info.numCommonUsers);
+        fbUserId = info.fbUserIdMin;
+        twUserId = info.twUserIdMin;
+
+        fbMessageId = info.fbMessageIdMin;
+        twMessageId = info.fbMessageIdMin;
+        duration = info.timeDurationInSecs;
+    }
+
+    public static void main(String args[]) throws Exception {
+
+        String controllerInstallDir = null;
+        if (args.length < 2) {
+            printUsage();
+            System.exit(1);
+        }
+
+        DataGenerator2 dataGenerator = new DataGenerator2(args);
+        dataGenerator.generateData();
+    }
+
+    public static void printUsage() {
+        System.out.println(" Error: Invalid number of arguments ");
+        System.out.println(" Usage :" + " DataGenerator <path to configuration file> <partition name> ");
+    }
+
+    public void generateData() throws IOException {
+        generateFacebookOnlyUsers(numFbOnlyUsers);
+        generateTwitterOnlyUsers(numTwOnlyUsers);
+        generateCommonUsers();
+        System.out.println("Partition :" + partition.getTargetPartition().getName() + " finished");
+    }
+
+    public void getFacebookUser(String usernameSuffix) {
+        String suggestedName = randNameGen.getRandomName();
+        String[] nameComponents = suggestedName.split(" ");
+        String name = nameComponents[0] + nameComponents[1];
+        if (usernameSuffix != null) {
+            name = name + usernameSuffix;
+        }
+        String alias = nameComponents[0];
+        String userSince = randDateGen.getNextRandomDatetime().toString();
+        int numFriends = random.nextInt(25);
+        int[] friendIds = RandomUtil.getKFromN(numFriends, (numFbOnlyUsers + numCommonUsers));
+        Employment emp = randEmpGen.getRandomEmployment();
+        fbUser.reset(fbUserId++, alias, name, userSince, friendIds, emp);
+    }
+
+    public void getTwitterUser(String usernameSuffix) {
+        String suggestedName = randNameGen.getRandomName();
+        String[] nameComponents = suggestedName.split(" ");
+        String screenName = nameComponents[0] + nameComponents[1] + randNameGen.getRandomNameSuffix();
+        String name = suggestedName;
+        if (usernameSuffix != null) {
+            name = name + usernameSuffix;
+        }
+        int numFriends = random.nextInt((int) (100)); // draw from Zipfian
+        int statusesCount = random.nextInt(500); // draw from Zipfian
+        int followersCount = random.nextInt((int) (200));
+        twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+        twUserId++;
+    }
+
+    public void getCorrespondingTwitterUser(FacebookUser fbUser) {
+        String screenName = fbUser.getName().substring(0, fbUser.getName().lastIndexOf(commonUserFbSuffix))
+                + commonUserTwSuffix;
+        String name = screenName.split(" ")[0];
+        int numFriends = random.nextInt((int) ((numTwOnlyUsers + numCommonUsers)));
+        int statusesCount = random.nextInt(500); // draw from Zipfian
+        int followersCount = random.nextInt((int) (numTwOnlyUsers + numCommonUsers));
+        twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+    }
+
+    public static class RandomDateGenerator {
+
+        private final Date startDate;
+        private final Date endDate;
+        private final Random random = new Random();
+        private final int yearDifference;
+        private Date workingDate;
+        private Date recentDate;
+        private DateTime dateTime;
+
+        public RandomDateGenerator(Date startDate, Date endDate) {
+            this.startDate = startDate;
+            this.endDate = endDate;
+            yearDifference = endDate.getYear() - startDate.getYear() + 1;
+            workingDate = new Date();
+            recentDate = new Date();
+            dateTime = new DateTime();
+        }
+
+        public Date getStartDate() {
+            return startDate;
+        }
+
+        public Date getEndDate() {
+            return endDate;
+        }
+
+        public Date getNextRandomDate() {
+            int year = random.nextInt(yearDifference) + startDate.getYear();
+            int month;
+            int day;
+            if (year == endDate.getYear()) {
+                month = random.nextInt(endDate.getMonth()) + 1;
+                if (month == endDate.getMonth()) {
+                    day = random.nextInt(endDate.getDay()) + 1;
+                } else {
+                    day = random.nextInt(28) + 1;
+                }
+            } else {
+                month = random.nextInt(12) + 1;
+                day = random.nextInt(28) + 1;
+            }
+            workingDate.reset(month, day, year);
+            return workingDate;
+        }
+
+        public DateTime getNextRandomDatetime() {
+            Date randomDate = getNextRandomDate();
+            dateTime.reset(randomDate);
+            return dateTime;
+        }
+
+        public Date getNextRecentDate(Date date) {
+            int year = date.getYear()
+                    + (date.getYear() == endDate.getYear() ? 0 : random.nextInt(endDate.getYear() - date.getYear()));
+            int month = (year == endDate.getYear()) ? date.getMonth() == endDate.getMonth() ? (endDate.getMonth())
+                    : (date.getMonth() + random.nextInt(endDate.getMonth() - date.getMonth())) : random.nextInt(12) + 1;
+
+            int day = (year == endDate.getYear()) ? month == endDate.getMonth() ? date.getDay() == endDate.getDay() ? endDate
+                    .getDay() : date.getDay() + random.nextInt(endDate.getDay() - date.getDay())
+                    : random.nextInt(28) + 1
+                    : random.nextInt(28) + 1;
+            recentDate.reset(month, day, year);
+            return recentDate;
+        }
+
+        public static void main(String args[]) throws Exception {
+            Date date = new Date(2, 20, 2012);
+            RandomDateGenerator dgen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+            while (true) {
+                Date nextDate = dgen.getNextRandomDate();
+                if (nextDate.getDay() == 0) {
+                    throw new Exception("invalid date " + nextDate);
+                }
+
+                // System.out.println(" original date: " + date);
+                System.out.println(nextDate);
+            }
+        }
+    }
+
+    public static class DateTime extends Date {
+
+        private String hour = "10";
+        private String min = "10";
+        private String sec = "00";
+        private long chrononTime;
+
+        public DateTime(int month, int day, int year, String hour, String min, String sec) {
+            super(month, day, year);
+            this.hour = hour;
+            this.min = min;
+            this.sec = sec;
+            chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+                    Integer.parseInt(sec)).getTime();
+        }
+
+        public void reset(int month, int day, int year, String hour, String min, String sec) {
+            super.setDay(month);
+            super.setDay(day);
+            super.setYear(year);
+            this.hour = hour;
+            this.min = min;
+            this.sec = sec;
+            chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+                    Integer.parseInt(sec)).getTime();
+        }
+
+        public DateTime() {
+        }
+
+        public DateTime(Date date) {
+            super(date.getMonth(), date.getDay(), date.getYear());
+        }
+
+        public void reset(Date date) {
+            reset(date.getMonth(), date.getDay(), date.getYear());
+        }
+
+        public DateTime(Date date, int hour, int min, int sec) {
+            super(date.getMonth(), date.getDay(), date.getYear());
+            this.hour = (hour < 10) ? "0" : "" + hour;
+            this.min = (min < 10) ? "0" : "" + min;
+            this.sec = (sec < 10) ? "0" : "" + sec;
+        }
+
+        public long getChrononTime() {
+            return chrononTime;
+        }
+
+        public String getHour() {
+            return hour;
+        }
+
+        public String getMin() {
+            return min;
+        }
+
+        public String getSec() {
+            return sec;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("datetime");
+            builder.append("(\"");
+            builder.append(super.getYear());
+            builder.append("-");
+            builder.append(super.getMonth() < 10 ? "0" + super.getMonth() : super.getMonth());
+            builder.append("-");
+            builder.append(super.getDay() < 10 ? "0" + super.getDay() : super.getDay());
+            builder.append("T");
+            builder.append(hour + ":" + min + ":" + sec);
+            builder.append("\")");
+            return builder.toString();
+        }
+    }
+
+    public static class Message {
+
+        private char[] message = new char[500];
+        private List<String> referredTopics;
+        private int length;
+
+        public Message(char[] m, List<String> referredTopics) {
+            System.arraycopy(m, 0, message, 0, m.length);
+            length = m.length;
+            this.referredTopics = referredTopics;
+        }
+
+        public Message() {
+            referredTopics = new ArrayList<String>();
+            length = 0;
+        }
+
+        public char[] getMessage() {
+            return message;
+        }
+
+        public List<String> getReferredTopics() {
+            return referredTopics;
+        }
+
+        public void reset(char[] m, int offset, int length, List<String> referredTopics) {
+            System.arraycopy(m, offset, message, 0, length);
+            this.length = length;
+            this.referredTopics = referredTopics;
+        }
+
+        public int getLength() {
+            return length;
+        }
+
+        public char charAt(int index) {
+            return message[index];
+        }
+
+    }
+
+    public static class Point {
+
+        private float latitude;
+        private float longitude;
+
+        public float getLatitude() {
+            return latitude;
+        }
+
+        public float getLongitude() {
+            return longitude;
+        }
+
+        public Point(float latitude, float longitude) {
+            this.latitude = latitude;
+            this.longitude = longitude;
+        }
+
+        public void reset(float latitude, float longitude) {
+            this.latitude = latitude;
+            this.longitude = longitude;
+        }
+
+        public Point() {
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("point(\"" + latitude + "," + longitude + "\")");
+            return builder.toString();
+        }
+    }
+
+    public static class RandomNameGenerator {
+
+        private String[] firstNames;
+        private String[] lastNames;
+
+        private final Random random = new Random();
+
+        private final String[] connectors = new String[] { "_", "#", "$", "@" };
+
+        public RandomNameGenerator(String firstNameFilePath, String lastNameFilePath) throws IOException {
+            firstNames = FileUtil.listyFile(new File(firstNameFilePath)).toArray(new String[] {});
+            lastNames = FileUtil.listyFile(new File(lastNameFilePath)).toArray(new String[] {});
+        }
+
+        public RandomNameGenerator(String[] firstNames, String[] lastNames) {
+            this.firstNames = firstNames;
+            this.lastNames = lastNames;
+        }
+
+        public String getRandomName() {
+            String name;
+            name = getSuggestedName();
+            return name;
+
+        }
+
+        private String getSuggestedName() {
+            int firstNameIndex = random.nextInt(firstNames.length);
+            int lastNameIndex = random.nextInt(lastNames.length);
+            String suggestedName = firstNames[firstNameIndex] + " " + lastNames[lastNameIndex];
+            return suggestedName;
+        }
+
+        public String getRandomNameSuffix() {
+            return connectors[random.nextInt(connectors.length)] + random.nextInt(1000);
+        }
+    }
+
+    public static class RandomMessageGenerator {
+
+        private final MessageTemplate messageTemplate;
+
+        public RandomMessageGenerator(String vendorFilePath, String jargonFilePath) throws IOException {
+            List<String> vendors = FileUtil.listyFile(new File(vendorFilePath));
+            List<String> jargon = FileUtil.listyFile(new File(jargonFilePath));
+            this.messageTemplate = new MessageTemplate(vendors, jargon);
+        }
+
+        public RandomMessageGenerator(String[] vendors, String[] jargon) {
+            List<String> vendorList = new ArrayList<String>();
+            for (String v : vendors) {
+                vendorList.add(v);
+            }
+            List<String> jargonList = new ArrayList<String>();
+            for (String j : jargon) {
+                jargonList.add(j);
+            }
+            this.messageTemplate = new MessageTemplate(vendorList, jargonList);
+        }
+
+        public Message getNextRandomMessage() {
+            return messageTemplate.getNextMessage();
+        }
+    }
+
+    public static class AbstractMessageTemplate {
+
+        protected final Random random = new Random();
+
+        protected String[] positiveVerbs = new String[] { "like", "love" };
+        protected String[] negativeVerbs = new String[] { "dislike", "hate", "can't stand" };
+
+        protected String[] negativeAdjectives = new String[] { "horrible", "bad", "terrible", "OMG" };
+        protected String[] postiveAdjectives = new String[] { "good", "awesome", "amazing", "mind-blowing" };
+
+        protected String[] otherWords = new String[] { "the", "its" };
+    }
+
+    public static class MessageTemplate extends AbstractMessageTemplate {
+
+        private List<String> vendors;
+        private List<String> jargon;
+        private CharBuffer buffer;
+        private List<String> referredTopics;
+        private Message message = new Message();
+
+        public MessageTemplate(List<String> vendors, List<String> jargon) {
+            this.vendors = vendors;
+            this.jargon = jargon;
+            buffer = CharBuffer.allocate(2500);
+            referredTopics = new ArrayList<String>();
+        }
+
+        public Message getNextMessage() {
+            buffer.position(0);
+            buffer.limit(2500);
+            referredTopics.clear();
+            boolean isPositive = random.nextBoolean();
+            String[] verbArray = isPositive ? positiveVerbs : negativeVerbs;
+            String[] adjectiveArray = isPositive ? postiveAdjectives : negativeAdjectives;
+            String verb = verbArray[random.nextInt(verbArray.length)];
+            String adjective = adjectiveArray[random.nextInt(adjectiveArray.length)];
+
+            buffer.put(" ");
+            buffer.put(verb);
+            buffer.put(" ");
+            String vendor = vendors.get(random.nextInt(vendors.size()));
+            referredTopics.add(vendor);
+            buffer.append(vendor);
+            buffer.append(" ");
+            buffer.append(otherWords[random.nextInt(otherWords.length)]);
+            buffer.append(" ");
+            String jargonTerm = jargon.get(random.nextInt(jargon.size()));
+            referredTopics.add(jargonTerm);
+            buffer.append(jargonTerm);
+            buffer.append(" is ");
+            buffer.append(adjective);
+            if (random.nextBoolean()) {
+                buffer.append(isPositive ? ":)" : ":(");
+            }
+
+            buffer.flip();
+            message.reset(buffer.array(), 0, buffer.limit(), referredTopics);
+            return message;
+        }
+    }
+
+    public static class RandomUtil {
+
+        public static Random random = new Random();
+
+        public static int[] getKFromN(int k, int n) {
+            int[] result = new int[k];
+            int cnt = 0;
+            HashSet<Integer> values = new HashSet<Integer>();
+            while (cnt < k) {
+                int val = random.nextInt(n + 1);
+                if (values.contains(val)) {
+                    continue;
+                }
+
+                result[cnt++] = val;
+                values.add(val);
+            }
+            return result;
+        }
+    }
+
+    public static class FileUtil {
+
+        public static List<String> listyFile(File file) throws IOException {
+
+            BufferedReader reader = new BufferedReader(new FileReader(file));
+            String line;
+            List<String> list = new ArrayList<String>();
+            while (true) {
+                line = reader.readLine();
+                if (line == null) {
+                    break;
+                }
+                list.add(line);
+            }
+            return list;
+        }
+
+        public static FileAppender getFileAppender(String filePath, boolean createIfNotExists, boolean overwrite)
+                throws IOException {
+            return new FileAppender(filePath, createIfNotExists, overwrite);
+        }
+    }
+
+    public static class FileAppender {
+
+        private final BufferedWriter writer;
+
+        public FileAppender(String filePath, boolean createIfNotExists, boolean overwrite) throws IOException {
+            File file = new File(filePath);
+            if (!file.exists()) {
+                if (createIfNotExists) {
+                    new File(file.getParent()).mkdirs();
+                } else {
+                    throw new IOException("path " + filePath + " does not exists");
+                }
+            }
+            this.writer = new BufferedWriter(new FileWriter(file, !overwrite));
+        }
+
+        public void appendToFile(String content) throws IOException {
+            writer.append(content);
+            writer.append("\n");
+        }
+
+        public void close() throws IOException {
+            writer.close();
+        }
+    }
+
+    public static class RandomEmploymentGenerator {
+
+        private final int percentEmployed;
+        private final Random random = new Random();
+        private final RandomDateGenerator randDateGen;
+        private final List<String> organizations;
+        private Employment emp;
+
+        public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String orgListPath)
+                throws IOException {
+            this.percentEmployed = percentEmployed;
+            this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+            organizations = FileUtil.listyFile(new File(orgListPath));
+            emp = new Employment();
+        }
+
+        public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String[] orgList) {
+            this.percentEmployed = percentEmployed;
+            this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+            organizations = new ArrayList<String>();
+            for (String org : orgList) {
+                organizations.add(org);
+            }
+            emp = new Employment();
+        }
+
+        public Employment getRandomEmployment() {
+            int empployed = random.nextInt(100) + 1;
+            boolean isEmployed = false;
+            if (empployed <= percentEmployed) {
+                isEmployed = true;
+            }
+            Date startDate = randDateGen.getNextRandomDate();
+            Date endDate = null;
+            if (!isEmployed) {
+                endDate = randDateGen.getNextRecentDate(startDate);
+            }
+            String org = organizations.get(random.nextInt(organizations.size()));
+            emp.reset(org, startDate, endDate);
+            return emp;
+        }
+    }
+
+    public static class RandomLocationGenerator {
+
+        private Random random = new Random();
+
+        private final int beginLat;
+        private final int endLat;
+        private final int beginLong;
+        private final int endLong;
+
+        private Point point;
+
+        public RandomLocationGenerator(int beginLat, int endLat, int beginLong, int endLong) {
+            this.beginLat = beginLat;
+            this.endLat = endLat;
+            this.beginLong = beginLong;
+            this.endLong = endLong;
+            this.point = new Point();
+        }
+
+        public Point getRandomPoint() {
+            int latMajor = beginLat + random.nextInt(endLat - beginLat);
+            int latMinor = random.nextInt(100);
+            float latitude = latMajor + ((float) latMinor) / 100;
+
+            int longMajor = beginLong + random.nextInt(endLong - beginLong);
+            int longMinor = random.nextInt(100);
+            float longitude = longMajor + ((float) longMinor) / 100;
+
+            point.reset(latitude, longitude);
+            return point;
+        }
+
+    }
+
+    public static class PartitionConfiguration {
+
+        private final TargetPartition targetPartition;
+        private final SourcePartition sourcePartition;
+
+        public PartitionConfiguration(SourcePartition sourcePartition, TargetPartition targetPartition) {
+            this.sourcePartition = sourcePartition;
+            this.targetPartition = targetPartition;
+        }
+
+        public TargetPartition getTargetPartition() {
+            return targetPartition;
+        }
+
+        public SourcePartition getSourcePartition() {
+            return sourcePartition;
+        }
+
+    }
+
+    public static class SourcePartition {
+
+        private final String name;
+        private final String host;
+        private final String path;
+
+        public SourcePartition(String name, String host, String path) {
+            this.name = name;
+            this.host = host;
+            this.path = path;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getHost() {
+            return host;
+        }
+
+        public String getPath() {
+            return path;
+        }
+    }
+
+    public static class TargetPartition {
+        private final String name;
+        private final String host;
+        private final String path;
+        private final int fbUserKeyMin;
+        private final int fbUserKeyMax;
+        private final int twUserKeyMin;
+        private final int twUserKeyMax;
+        private final int fbMessageIdMin;
+        private final int fbMessageIdMax;
+        private final int twMessageIdMin;
+        private final int twMessageIdMax;
+        private final int commonUsers;
+
+        public TargetPartition(String partitionName, String host, String path, int fbUserKeyMin, int fbUserKeyMax,
+                int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+                int twMessageIdMax, int commonUsers) {
+            this.name = partitionName;
+            this.host = host;
+            this.path = path;
+            this.fbUserKeyMin = fbUserKeyMin;
+            this.fbUserKeyMax = fbUserKeyMax;
+            this.twUserKeyMin = twUserKeyMin;
+            this.twUserKeyMax = twUserKeyMax;
+            this.twMessageIdMin = twMessageIdMin;
+            this.twMessageIdMax = twMessageIdMax;
+            this.fbMessageIdMin = fbMessageIdMin;
+            this.fbMessageIdMax = fbMessageIdMax;
+            this.commonUsers = commonUsers;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append(name);
+            builder.append(" ");
+            builder.append(host);
+            builder.append("\n");
+            builder.append(path);
+            builder.append("\n");
+            builder.append("fbUser:key:min");
+            builder.append(fbUserKeyMin);
+
+            builder.append("\n");
+            builder.append("fbUser:key:max");
+            builder.append(fbUserKeyMax);
+
+            builder.append("\n");
+            builder.append("twUser:key:min");
+            builder.append(twUserKeyMin);
+
+            builder.append("\n");
+            builder.append("twUser:key:max");
+            builder.append(twUserKeyMax);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:min");
+            builder.append(fbMessageIdMin);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:max");
+            builder.append(fbMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:min");
+            builder.append(twMessageIdMin);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("commonUsers");
+            builder.append(commonUsers);
+
+            return new String(builder);
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getHost() {
+            return host;
+        }
+
+        public int getFbUserKeyMin() {
+            return fbUserKeyMin;
+        }
+
+        public int getFbUserKeyMax() {
+            return fbUserKeyMax;
+        }
+
+        public int getTwUserKeyMin() {
+            return twUserKeyMin;
+        }
+
+        public int getTwUserKeyMax() {
+            return twUserKeyMax;
+        }
+
+        public int getFbMessageIdMin() {
+            return fbMessageIdMin;
+        }
+
+        public int getFbMessageIdMax() {
+            return fbMessageIdMax;
+        }
+
+        public int getTwMessageIdMin() {
+            return twMessageIdMin;
+        }
+
+        public int getTwMessageIdMax() {
+            return twMessageIdMax;
+        }
+
+        public int getCommonUsers() {
+            return commonUsers;
+        }
+
+        public String getPath() {
+            return path;
+        }
+    }
+
+    public static class Employment {
+
+        private String organization;
+        private Date startDate;
+        private Date endDate;
+
+        public Employment(String organization, Date startDate, Date endDate) {
+            this.organization = organization;
+            this.startDate = startDate;
+            this.endDate = endDate;
+        }
+
+        public Employment() {
+        }
+
+        public void reset(String organization, Date startDate, Date endDate) {
+            this.organization = organization;
+            this.startDate = startDate;
+            this.endDate = endDate;
+        }
+
+        public String getOrganization() {
+            return organization;
+        }
+
+        public Date getStartDate() {
+            return startDate;
+        }
+
+        public Date getEndDate() {
+            return endDate;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder("");
+            builder.append("{");
+            builder.append("\"organization-name\":");
+            builder.append("\"" + organization + "\"");
+            builder.append(",");
+            builder.append("\"start-date\":");
+            builder.append(startDate);
+            if (endDate != null) {
+                builder.append(",");
+                builder.append("\"end-date\":");
+                builder.append(endDate);
+            }
+            builder.append("}");
+            return new String(builder);
+        }
+
+    }
+
+    public static class FacebookMessage {
+
+        private int messageId;
+        private int authorId;
+        private int inResponseTo;
+        private Point senderLocation;
+        private Message message;
+
+        public int getMessageId() {
+            return messageId;
+        }
+
+        public int getAuthorID() {
+            return authorId;
+        }
+
+        public Point getSenderLocation() {
+            return senderLocation;
+        }
+
+        public Message getMessage() {
+            return message;
+        }
+
+        public int getInResponseTo() {
+            return inResponseTo;
+        }
+
+        public FacebookMessage() {
+
+        }
+
+        public FacebookMessage(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+            this.messageId = messageId;
+            this.authorId = authorId;
+            this.inResponseTo = inResponseTo;
+            this.senderLocation = senderLocation;
+            this.message = message;
+        }
+
+        public void reset(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+            this.messageId = messageId;
+            this.authorId = authorId;
+            this.inResponseTo = inResponseTo;
+            this.senderLocation = senderLocation;
+            this.message = message;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"message-id\":");
+            builder.append(messageId);
+            builder.append(",");
+            builder.append("\"author-id\":");
+            builder.append(authorId);
+            builder.append(",");
+            builder.append("\"in-response-to\":");
+            builder.append(inResponseTo);
+            builder.append(",");
+            builder.append("\"sender-location\":");
+            builder.append(senderLocation);
+            builder.append(",");
+            builder.append("\"message\":");
+            builder.append("\"");
+            for (int i = 0; i < message.getLength(); i++) {
+                builder.append(message.charAt(i));
+            }
+            builder.append("\"");
+            builder.append("}");
+            return new String(builder);
+        }
+    }
+
+    public static class FacebookUser {
+
+        private int id;
+        private String alias;
+        private String name;
+        private String userSince;
+        private int[] friendIds;
+        private Employment employment;
+
+        public FacebookUser() {
+
+        }
+
+        public FacebookUser(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+            this.id = id;
+            this.alias = alias;
+            this.name = name;
+            this.userSince = userSince;
+            this.friendIds = friendIds;
+            this.employment = employment;
+        }
+
+        public int getId() {
+            return id;
+        }
+
+        public String getAlias() {
+            return alias;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getUserSince() {
+            return userSince;
+        }
+
+        public int[] getFriendIds() {
+            return friendIds;
+        }
+
+        public Employment getEmployment() {
+            return employment;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"id\":" + id);
+            builder.append(",");
+            builder.append("\"alias\":" + "\"" + alias + "\"");
+            builder.append(",");
+            builder.append("\"name\":" + "\"" + name + "\"");
+            builder.append(",");
+            builder.append("\"user-since\":" + userSince);
+            builder.append(",");
+            builder.append("\"friend-ids\":");
+            builder.append("{{");
+            for (int i = 0; i < friendIds.length; i++) {
+                builder.append(friendIds[i]);
+                builder.append(",");
+            }
+            if (friendIds.length > 0) {
+                builder.deleteCharAt(builder.lastIndexOf(","));
+            }
+            builder.append("}}");
+            builder.append(",");
+            builder.append("\"employment\":");
+            builder.append("[");
+            builder.append(employment.toString());
+            builder.append("]");
+            builder.append("}");
+            return builder.toString();
+        }
+
+        public void setId(int id) {
+            this.id = id;
+        }
+
+        public void setAlias(String alias) {
+            this.alias = alias;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public void setUserSince(String userSince) {
+            this.userSince = userSince;
+        }
+
+        public void setFriendIds(int[] friendIds) {
+            this.friendIds = friendIds;
+        }
+
+        public void setEmployment(Employment employment) {
+            this.employment = employment;
+        }
+
+        public void reset(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+            this.id = id;
+            this.alias = alias;
+            this.name = name;
+            this.userSince = userSince;
+            this.friendIds = friendIds;
+            this.employment = employment;
+        }
+    }
+
+    public static class TweetMessage {
+
+        private String tweetid;
+        private TwitterUser user;
+        private Point senderLocation;
+        private DateTime sendTime;
+        private List<String> referredTopics;
+        private Message messageText;
+
+        public TweetMessage() {
+
+        }
+
+        public TweetMessage(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+                List<String> referredTopics, Message messageText) {
+            this.tweetid = tweetid;
+            this.user = user;
+            this.senderLocation = senderLocation;
+            this.sendTime = sendTime;
+            this.referredTopics = referredTopics;
+            this.messageText = messageText;
+        }
+
+        public void reset(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+                List<String> referredTopics, Message messageText) {
+            this.tweetid = tweetid;
+            this.user = user;
+            this.senderLocation = senderLocation;
+            this.sendTime = sendTime;
+            this.referredTopics = referredTopics;
+            this.messageText = messageText;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"tweetid\":");
+            builder.append("\"" + tweetid + "\"");
+            builder.append(",");
+            builder.append("\"user\":");
+            builder.append(user);
+            builder.append(",");
+            builder.append("\"sender-location\":");
+            builder.append(senderLocation);
+            builder.append(",");
+            builder.append("\"send-time\":");
+            builder.append(sendTime);
+            builder.append(",");
+            builder.append("\"referred-topics\":");
+            builder.append("{{");
+            for (String topic : referredTopics) {
+                builder.append("\"" + topic + "\"");
+                builder.append(",");
+            }
+            if (referredTopics.size() > 0) {
+                builder.deleteCharAt(builder.lastIndexOf(","));
+            }
+            builder.append("}}");
+            builder.append(",");
+            builder.append("\"message-text\":");
+            builder.append("\"");
+            for (int i = 0; i < messageText.getLength(); i++) {
+                builder.append(messageText.charAt(i));
+            }
+            builder.append("\"");
+            builder.append("}");
+            return new String(builder);
+        }
+
+        public String getTweetid() {
+            return tweetid;
+        }
+
+        public void setTweetid(String tweetid) {
+            this.tweetid = tweetid;
+        }
+
+        public TwitterUser getUser() {
+            return user;
+        }
+
+        public void setUser(TwitterUser user) {
+            this.user = user;
+        }
+
+        public Point getSenderLocation() {
+            return senderLocation;
+        }
+
+        public void setSenderLocation(Point senderLocation) {
+            this.senderLocation = senderLocation;
+        }
+
+        public DateTime getSendTime() {
+            return sendTime;
+        }
+
+        public void setSendTime(DateTime sendTime) {
+            this.sendTime = sendTime;
+        }
+
+        public List<String> getReferredTopics() {
+            return referredTopics;
+        }
+
+        public void setReferredTopics(List<String> referredTopics) {
+            this.referredTopics = referredTopics;
+        }
+
+        public Message getMessageText() {
+            return messageText;
+        }
+
+        public void setMessageText(Message messageText) {
+            this.messageText = messageText;
+        }
+
+    }
+
+    public static class TwitterUser {
+
+        private String screenName;
+        private String lang = "en";
+        private int friendsCount;
+        private int statusesCount;
+        private String name;
+        private int followersCount;
+
+        public TwitterUser() {
+
+        }
+
+        public TwitterUser(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+            this.screenName = screenName;
+            this.friendsCount = friendsCount;
+            this.statusesCount = statusesCount;
+            this.name = name;
+            this.followersCount = followersCount;
+        }
+
+        public void reset(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+            this.screenName = screenName;
+            this.friendsCount = friendsCount;
+            this.statusesCount = statusesCount;
+            this.name = name;
+            this.followersCount = followersCount;
+        }
+
+        public String getScreenName() {
+            return screenName;
+        }
+
+        public int getFriendsCount() {
+            return friendsCount;
+        }
+
+        public int getStatusesCount() {
+            return statusesCount;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public int getFollowersCount() {
+            return followersCount;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("{");
+            builder.append("\"screen-name\":" + "\"" + screenName + "\"");
+            builder.append(",");
+            builder.append("\"lang\":" + "\"" + lang + "\"");
+            builder.append(",");
+            builder.append("\"friends_count\":" + friendsCount);
+            builder.append(",");
+            builder.append("\"statuses_count\":" + statusesCount);
+            builder.append(",");
+            builder.append("\"name\":" + "\"" + name + "\"");
+            builder.append(",");
+            builder.append("\"followers_count\":" + followersCount);
+            builder.append("}");
+            return builder.toString();
+        }
+
+    }
+
+    public static class DistributionHandler {
+
+        private final ZipfGenerator zipfGen;
+        private final int totalUsers;
+        private final int totalMessages;
+        private Random random = new Random();
+
+        public DistributionHandler(int totalMessages, double skew, int totalNumUsers) {
+            zipfGen = new ZipfGenerator(totalMessages, skew);
+            totalUsers = totalNumUsers;
+            this.totalMessages = totalMessages;
+        }
+
+        public int getFromDistribution(int rank) {
+            double prob = zipfGen.getProbability(rank);
+            int numMessages = (int) (prob * totalMessages);
+            return numMessages;
+        }
+
+        public static void main(String args[]) {
+            int totalMessages = 1000 * 4070;
+            double skew = 0.5;
+            int totalUsers = 4070;
+            DistributionHandler d = new DistributionHandler(totalMessages, skew, totalUsers);
+            int sum = 0;
+            for (int i = totalUsers; i >= 1; i--) {
+                float contrib = d.getFromDistribution(i);
+                sum += contrib;
+                System.out.println(i + ":" + contrib);
+            }
+
+            System.out.println("SUM" + ":" + sum);
+
+        }
+    }
+
+    public static class ZipfGenerator {
+
+        private Random rnd = new Random(System.currentTimeMillis());
+        private int size;
+        private double skew;
+        private double bottom = 0;
+
+        public ZipfGenerator(int size, double skew) {
+            this.size = size;
+            this.skew = skew;
+            for (int i = 1; i < size; i++) {
+                this.bottom += (1 / Math.pow(i, this.skew));
+            }
+        }
+
+        // the next() method returns an rank id. The frequency of returned rank
+        // ids are follows Zipf distribution.
+        public int next() {
+            int rank;
+            double friquency = 0;
+            double dice;
+            rank = rnd.nextInt(size);
+            friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+            dice = rnd.nextDouble();
+            while (!(dice < friquency)) {
+                rank = rnd.nextInt(size);
+                friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+                dice = rnd.nextDouble();
+            }
+            return rank;
+        }
+
+        // This method returns a probability that the given rank occurs.
+        public double getProbability(int rank) {
+            return (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+        }
+
+        public static void main(String[] args) throws IOException {
+            int total = (int) (3.7 * 1000 * 1000);
+            int skew = 2;
+            int numUsers = 1000 * 1000;
+            /*
+             * if (args.length != 2) { System.out.println("usage:" +
+             * "./zipf size skew"); System.exit(-1); }
+             */
+            BufferedWriter buf = new BufferedWriter(new FileWriter(new File("/tmp/zip_output")));
+            ZipfGenerator zipf = new ZipfGenerator(total, skew);
+            double sum = 0;
+            for (int i = 1; i <= numUsers; i++) {
+                double prob = zipf.getProbability(i);
+                double contribution = (double) (prob * total);
+                String contrib = i + ":" + contribution;
+                buf.write(contrib);
+                buf.write("\n");
+                System.out.println(contrib);
+                sum += contribution;
+            }
+            System.out.println("sum is :" + sum);
+        }
+    }
+
+    public static class PartitionElement implements ILibraryElement {
+        private final String name;
+        private final String host;
+        private final int fbUserKeyMin;
+        private final int fbUserKeyMax;
+        private final int twUserKeyMin;
+        private final int twUserKeyMax;
+        private final int fbMessageIdMin;
+        private final int fbMessageIdMax;
+        private final int twMessageIdMin;
+        private final int twMessageIdMax;
+
+        public PartitionElement(String partitionName, String host, int fbUserKeyMin, int fbUserKeyMax,
+                int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+                int twMessageIdMax) {
+            this.name = partitionName;
+            this.host = host;
+            this.fbUserKeyMin = fbUserKeyMin;
+            this.fbUserKeyMax = fbUserKeyMax;
+            this.twUserKeyMin = twUserKeyMax;
+            this.twUserKeyMax = twUserKeyMax;
+            this.twMessageIdMin = twMessageIdMin;
+            this.twMessageIdMax = twMessageIdMax;
+            this.fbMessageIdMin = fbMessageIdMin;
+            this.fbMessageIdMax = fbMessageIdMax;
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append(name);
+            builder.append(" ");
+            builder.append(host);
+            builder.append("\n");
+            builder.append("fbUser:key:min");
+            builder.append(fbUserKeyMin);
+
+            builder.append("\n");
+            builder.append("fbUser:key:max");
+            builder.append(fbUserKeyMax);
+
+            builder.append("\n");
+            builder.append("twUser:key:min");
+            builder.append(twUserKeyMin);
+
+            builder.append("\n");
+            builder.append("twUser:key:max");
+            builder.append(twUserKeyMax);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:min");
+            builder.append(fbMessageIdMin);
+
+            builder.append("\n");
+            builder.append("fbMessage:key:max");
+            builder.append(fbMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:min");
+            builder.append(twMessageIdMin);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twMessageIdMax);
+
+            builder.append("\n");
+            builder.append("twMessage:key:max");
+            builder.append(twUserKeyMin);
+
+            return new String(builder);
+        }
+
+        @Override
+        public String getName() {
+            return "Partition";
+        }
+
+    }
+
+    interface ILibraryElement {
+
+        public enum ElementType {
+            PARTITION
+        }
+
+        public String getName();
+
+    }
+
+    public static class Configuration {
+
+        private final float numMB;
+        private final String unit;
+
+        private final List<SourcePartition> sourcePartitions;
+        private List<TargetPartition> targetPartitions;
+
+        public Configuration(float numMB, String unit, List<SourcePartition> partitions) throws IOException {
+            this.numMB = numMB;
+            this.unit = unit;
+            this.sourcePartitions = partitions;
+
+        }
+
+        public float getNumMB() {
+            return numMB;
+        }
+
+        public String getUnit() {
+            return unit;
+        }
+
+        public List<SourcePartition> getSourcePartitions() {
+            return sourcePartitions;
+        }
+
+        public List<TargetPartition> getTargetPartitions() {
+            return targetPartitions;
+        }
+
+        public void setTargetPartitions(List<TargetPartition> targetPartitions) {
+            this.targetPartitions = targetPartitions;
+        }
+
+    }
+
+    public static class XMLUtil {
+
+        public static void writeToXML(Configuration conf, String filePath) throws IOException,
+                ParserConfigurationException, TransformerException {
+
+            DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+
+            // root elements
+            Document doc = docBuilder.newDocument();
+            Element rootElement = doc.createElement("Partitions");
+            doc.appendChild(rootElement);
+
+            int index = 0;
+            for (TargetPartition partition : conf.getTargetPartitions()) {
+                writePartitionElement(conf.getSourcePartitions().get(index), partition, rootElement, doc);
+            }
+
+            TransformerFactory transformerFactory = TransformerFactory.newInstance();
+            Transformer transformer = transformerFactory.newTransformer();
+
+            transformer.setOutputProperty(OutputKeys.ENCODING, "utf-8");
+            transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "no");
+            transformer.setOutputProperty(OutputKeys.INDENT, "yes");
+            transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+
+            DOMSource source = new DOMSource(doc);
+            StreamResult result = new StreamResult(new File(filePath));
+
+            transformer.transform(source, result);
+
+        }
+
+        public static void writePartitionInfo(Configuration conf, String filePath) throws IOException {
+            BufferedWriter bw = new BufferedWriter(new FileWriter(filePath));
+            for (SourcePartition sp : conf.getSourcePartitions()) {
+                bw.write(sp.getHost() + ":" + sp.getName() + ":" + sp.getPath());
+                bw.write("\n");
+            }
+            bw.close();
+        }
+
+        public static Document getDocument(String filePath) throws Exception {
+            File inputFile = new File(filePath);
+            DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+            DocumentBuilder db = dbf.newDocumentBuilder();
+            Document doc = db.parse(inputFile);
+            doc.getDocumentElement().normalize();
+            return doc;
+        }
+
+        public static Configuration getConfiguration(String filePath) throws Exception {
+            Configuration conf = getConfiguration(getDocument(filePath));
+            PartitionMetrics metrics = new PartitionMetrics(conf.getNumMB(), conf.getUnit(), conf.getSourcePartitions()
+                    .size());
+            List<TargetPartition> targetPartitions = getTargetPartitions(metrics, conf.getSourcePartitions());
+            conf.setTargetPartitions(targetPartitions);
+            return conf;
+        }
+
+        public static Configuration getConfiguration(Document document) throws IOException {
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getChildNodes();
+            float size = Float.parseFloat(getStringValue((Element) nodeList, "size"));
+            String unit = getStringValue((Element) nodeList, "unit");
+            List<SourcePartition> sourcePartitions = getSourcePartitions(document);
+            return new Configuration(size, unit, sourcePartitions);
+        }
+
+        public static List<SourcePartition> getSourcePartitions(Document document) {
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getElementsByTagName("partition");
+            List<SourcePartition> sourcePartitions = new ArrayList<SourcePartition>();
+            for (int i = 0; i < nodeList.getLength(); i++) {
+                Node node = nodeList.item(i);
+                sourcePartitions.add(getSourcePartition((Element) node));
+            }
+            return sourcePartitions;
+        }
+
+        public static SourcePartition getSourcePartition(Element functionElement) {
+            String name = getStringValue(functionElement, "name");
+            String host = getStringValue(functionElement, "host");
+            String path = getStringValue(functionElement, "path");
+            SourcePartition sp = new SourcePartition(name, host, path);
+            return sp;
+        }
+
+        public static String getStringValue(Element element, String tagName) {
+            String textValue = null;
+            NodeList nl = element.getElementsByTagName(tagName);
+            if (nl != null && nl.getLength() > 0) {
+                Element el = (Element) nl.item(0);
+                textValue = el.getFirstChild().getNodeValue();
+            }
+            return textValue;
+        }
+
+        public static PartitionConfiguration getPartitionConfiguration(String filePath, String partitionName)
+                throws Exception {
+            PartitionConfiguration pconf = getPartitionConfigurations(getDocument(filePath), partitionName);
+            return pconf;
+        }
+
+        public static PartitionConfiguration getPartitionConfigurations(Document document, String partitionName)
+                throws IOException {
+
+            Element rootEle = document.getDocumentElement();
+            NodeList nodeList = rootEle.getElementsByTagName("Partition");
+
+            for (int i = 0; i < nodeList.getLength(); i++) {
+                Node node = nodeList.item(i);
+                Element nodeElement = (Element) node;
+                String name = getStringValue(nodeElement, "name");
+                if (!name.equalsIgnoreCase(partitionName)) {
+                    continue;
+                }
+                String host = getStringValue(nodeElement, "host");
+                String path = getStringValue(nodeElement, "path");
+
+                String fbUserKeyMin = getStringValue(nodeElement, "fbUserKeyMin");
+                String fbUserKeyMax = getStringValue(nodeElement, "fbUserKeyMax");
+                String twUserKeyMin = getStringValue(nodeElement, "twUserKeyMin");
+                String twUserKeyMax = getStringValue(nodeElement, "twUserKeyMax");
+                String fbMessageKeyMin = getStringValue(nodeElement, "fbMessageKeyMin");
+
+                String fbMessageKeyMax = getStringValue(nodeElement, "fbMessageKeyMax");
+                String twMessageKeyMin = getStringValue(nodeElement, "twMessageKeyMin");
+                String twMessageKeyMax = getStringValue(nodeElement, "twMessageKeyMax");
+                String numCommonUsers = getStringValue(nodeElement, "numCommonUsers");
+
+                SourcePartition sp = new SourcePartition(name, host, path);
+
+                TargetPartition tp = new TargetPartition(partitionName, host, path, Integer.parseInt(fbUserKeyMin),
+                        Integer.parseInt(fbUserKeyMax), Integer.parseInt(twUserKeyMin), Integer.parseInt(twUserKeyMax),
+                        Integer.parseInt(fbMessageKeyMin), Integer.parseInt(fbMessageKeyMax),
+                        Integer.parseInt(twMessageKeyMin), Integer.parseInt(twMessageKeyMax),
+                        Integer.parseInt(numCommonUsers));
+                PartitionConfiguration pc = new PartitionConfiguration(sp, tp);
+                return pc;
+            }
+            return null;
+        }
+
+        public static List<TargetPartition> getTargetPartitions(PartitionMetrics metrics,
+                List<SourcePartition> sourcePartitions) {
+            List<TargetPartition> partitions = new ArrayList<TargetPartition>();
+            int fbUserKeyMin = 1;
+            int twUserKeyMin = 1;
+            int fbMessageIdMin = 1;
+            int twMessageIdMin = 1;
+
+            for (SourcePartition sp : sourcePartitions) {
+                int fbUserKeyMax = fbUserKeyMin + metrics.getFbOnlyUsers() + metrics.getCommonUsers() - 1;
+                int twUserKeyMax = twUserKeyMin + metrics.getTwitterOnlyUsers() + metrics.getCommonUsers() - 1;
+
+                int fbMessageIdMax = fbMessageIdMin + metrics.getFbMessages() - 1;
+                int twMessageIdMax = twMessageIdMin + metrics.getTwMessages() - 1;
+                TargetPartition pe = new TargetPartition(sp.getName(), sp.getHost(), sp.getPath(), fbUserKeyMin,
+                        fbUserKeyMax, twUserKeyMin, twUserKeyMax, fbMessageIdMin, fbMessageIdMax, twMessageIdMin,
+                        twMessageIdMax, metrics.getCommonUsers());
+                partitions.add(pe);
+
+                fbUserKeyMin = fbUserKeyMax + 1;
+                twUserKeyMin = twUserKeyMax + 1;
+
+                fbMessageIdMin = fbMessageIdMax + 1;
+                twMessageIdMin = twMessageIdMax + 1;
+            }
+
+            return partitions;
+        }
+
+        public static void writePartitionElement(SourcePartition sourcePartition, TargetPartition partition,
+                Element rootElement, Document doc) {
+            // staff elements
+            Element pe = doc.createElement("Partition");
+            rootElement.appendChild(pe);
+
+            // name element
+            Element name = doc.createElement("name");
+            name.appendChild(doc.createTextNode("" + partition.getName()));
+            pe.appendChild(name);
+
+            // host element
+            Element host = doc.createElement("host");
+            host.appendChild(doc.createTextNode("" + partition.getHost()));
+            pe.appendChild(host);
+
+            // path element
+            Element path = doc.createElement("path");
+            path.appendChild(doc.createTextNode("" + partition.getPath()));
+            pe.appendChild(path);
+
+            // fbUserKeyMin element
+            Element fbUserKeyMin = doc.createElement("fbUserKeyMin");
+            fbUserKeyMin.appendChild(doc.createTextNode("" + partition.getFbUserKeyMin()));
+            pe.appendChild(fbUserKeyMin);
+
+            // fbUserKeyMax element
+            Element fbUserKeyMax = doc.createElement("fbUserKeyMax");
+            fbUserKeyMax.appendChild(doc.createTextNode("" + partition.getFbUserKeyMax()));
+            pe.appendChild(fbUserKeyMax);
+
+            // twUserKeyMin element
+            Element twUserKeyMin = doc.createElement("twUserKeyMin");
+            twUserKeyMin.appendChild(doc.createTextNode("" + partition.getTwUserKeyMin()));
+            pe.appendChild(twUserKeyMin);
+
+            // twUserKeyMax element
+            Element twUserKeyMax = doc.createElement("twUserKeyMax");
+            twUserKeyMax.appendChild(doc.createTextNode("" + partition.getTwUserKeyMax()));
+            pe.appendChild(twUserKeyMax);
+
+            // fbMessgeKeyMin element
+            Element fbMessageKeyMin = doc.createElement("fbMessageKeyMin");
+            fbMessageKeyMin.appendChild(doc.createTextNode("" + partition.getFbMessageIdMin()));
+            pe.appendChild(fbMessageKeyMin);
+
+            // fbMessgeKeyMin element
+            Element fbMessageKeyMax = doc.createElement("fbMessageKeyMax");
+            fbMessageKeyMax.appendChild(doc.createTextNode("" + partition.getFbMessageIdMax()));
+            pe.appendChild(fbMessageKeyMax);
+
+            // twMessgeKeyMin element
+            Element twMessageKeyMin = doc.createElement("twMessageKeyMin");
+            twMessageKeyMin.appendChild(doc.createTextNode("" + partition.getTwMessageIdMin()));
+            pe.appendChild(twMessageKeyMin);
+
+            // twMessgeKeyMin element
+            Element twMessageKeyMax = doc.createElement("twMessageKeyMax");
+            twMessageKeyMax.appendChild(doc.createTextNode("" + partition.getTwMessageIdMax()));
+            pe.appendChild(twMessageKeyMax);
+
+            // twMessgeKeyMin element
+            Element numCommonUsers = doc.createElement("numCommonUsers");
+            numCommonUsers.appendChild(doc.createTextNode("" + partition.getCommonUsers()));
+            pe.appendChild(numCommonUsers);
+
+        }
+
+        public static void main(String args[]) throws Exception {
+            String confFile = "/Users/rgrove1/work/research/asterix/icde/data-gen/conf/conf.xml";
+            String outputPath = "/Users/rgrove1/work/research/asterix/icde/data-gen/output/conf-output.xml";
+            Configuration conf = getConfiguration(confFile);
+            writeToXML(conf, outputPath);
+        }
+
+    }
+
+    public static class Date {
+
+        private int day;
+        private int month;
+        private int year;
+
+        public Date(int month, int day, int year) {
+            this.month = month;
+            this.day = day;
+            this.year = year;
+        }
+
+        public void reset(int month, int day, int year) {
+            this.month = month;
+            this.day = day;
+            this.year = year;
+        }
+
+        public int getDay() {
+            return day;
+        }
+
+        public int getMonth() {
+            return month;
+        }
+
+        public int getYear() {
+            return year;
+        }
+
+        public Date() {
+        }
+
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("date");
+            builder.append("(\"");
+            builder.append(year);
+            builder.append("-");
+            builder.append(month < 10 ? "0" + month : "" + month);
+            builder.append("-");
+            builder.append(day < 10 ? "0" + day : "" + day);
+            builder.append("\")");
+            return builder.toString();
+        }
+
+        public void setDay(int day) {
+            this.day = day;
+        }
+
+        public void setMonth(int month) {
+            this.month = month;
+        }
+
+        public void setYear(int year) {
+            this.year = year;
+        }
+    }
+
+    public static class PartitionMetrics {
+
+        private final int fbMessages;
+        private final int twMessages;
+
+        private final int fbOnlyUsers;
+        private final int twitterOnlyUsers;
+        private final int commonUsers;
+
+        public PartitionMetrics(float number, String unit, int numPartitions) throws IOException {
+
+            int factor = 0;
+            if (unit.equalsIgnoreCase("MB")) {
+                factor = 1024 * 1024;
+            } else if (unit.equalsIgnoreCase("GB")) {
+                factor = 1024 * 1024 * 1024;
+            } else if (unit.equalsIgnoreCase("TB")) {
+                factor = 1024 * 1024 * 1024 * 1024;
+            } else
+                throw new IOException("Invalid unit:" + unit);
+
+            fbMessages = (int) (((number * factor * 0.80) / 258) / numPartitions);
+            twMessages = (int) (fbMessages * 1.1 / 0.35);
+
+            fbOnlyUsers = (int) ((number * factor * 0.20 * 0.0043)) / numPartitions;
+            twitterOnlyUsers = (int) (0.25 * fbOnlyUsers);
+            commonUsers = (int) (0.1 * fbOnlyUsers);
+        }
+
+        public int getFbMessages() {
+            return fbMessages;
+        }
+
+        public int getTwMessages() {
+            return twMessages;
+        }
+
+        public int getFbOnlyUsers() {
+            return fbOnlyUsers;
+        }
+
+        public int getTwitterOnlyUsers() {
+            return twitterOnlyUsers;
+        }
+
+        public int getCommonUsers() {
+            return commonUsers;
+        }
+
+    }
+
+    public static String[] lastNames = { "Hoopengarner", "Harrow", "Gardner", "Blyant", "Best", "Buttermore", "Gronko",
+            "Mayers", "Countryman", "Neely", "Ruhl", "Taggart", "Bash", "Cason", "Hil", "Zalack", "Mingle", "Carr",
+            "Rohtin", "Wardle", "Pullman", "Wire", "Kellogg", "Hiles", "Keppel", "Bratton", "Sutton", "Wickes",
+            "Muller", "Friedline", "Llora", "Elizabeth", "Anderson", "Gaskins", "Rifler", "Vinsant", "Stanfield",
+            "Black", "Guest", "Hujsak", "Carter", "Weidemann", "Hays", "Patton", "Hayhurst", "Paynter", "Cressman",
+            "Fiddler", "Evans", "Sherlock", "Woodworth", "Jackson", "Bloise", "Schneider", "Ring", "Kepplinger",
+            "James", "Moon", "Bennett", "Bashline", "Ryals", "Zeal", "Christman", "Milliron", "Nash", "Ewing", "Camp",
+            "Mason", "Richardson", "Bowchiew", "Hahn", "Wilson", "Wood", "Toyley", "Williamson", "Lafortune", "Errett",
+            "Saltser", "Hirleman", "Brindle", "Newbiggin", "Ulery", "Lambert", "Shick", "Kuster", "Moore", "Finck",
+            "Powell", "Jolce", "Townsend", "Sauter", "Cowher", "Wolfe", "Cavalet", "Porter", "Laborde", "Ballou",
+            "Murray", "Stoddard", "Pycroft", "Milne", "King", "Todd", "Staymates", "Hall", "Romanoff", "Keilbach",
+            "Sandford", "Hamilton", "Fye", "Kline", "Weeks", "Mcelroy", "Mccullough", "Bryant", "Hill", "Moore",
+            "Ledgerwood", "Prevatt", "Eckert", "Read", "Hastings", "Doverspike", "Allshouse", "Bryan", "Mccallum",
+            "Lombardi", "Mckendrick", "Cattley", "Barkley", "Steiner", "Finlay", "Priebe", "Armitage", "Hall", "Elder",
+            "Erskine", "Hatcher", "Walker", "Pearsall", "Dunkle", "Haile", "Adams", "Miller", "Newbern", "Basinger",
+            "Fuhrer", "Brinigh", "Mench", "Blackburn", "Bastion", "Mccune", "Bridger", "Hynes", "Quinn", "Courtney",
+            "Geddinge", "Field", "Seelig", "Cable", "Earhart", "Harshman", "Roby", "Beals", "Berry", "Reed", "Hector",
+            "Pittman", "Haverrman", "Kalp", "Briner", "Joghs", "Cowart", "Close", "Wynne", "Harden", "Weldy",
+            "Stephenson", "Hildyard", "Moberly", "Wells", "Mackendoerfer", "Fisher", "Oppie", "Oneal", "Churchill",
+            "Keister", "Alice", "Tavoularis", "Fisher", "Hair", "Burns", "Veith", "Wile", "Fuller", "Fields", "Clark",
+            "Randolph", "Stone", "Mcclymonds", "Holtzer", "Donkin", "Wilkinson", "Rosensteel", "Albright", "Stahl",
+            "Fox", "Kadel", "Houser", "Hanseu", "Henderson", "Davis", "Bicknell", "Swain", "Mercer", "Holdeman",
+            "Enderly", "Caesar", "Margaret", "Munshower", "Elless", "Lucy", "Feufer", "Schofield", "Graham",
+            "Blatenberger", "Benford", "Akers", "Campbell", "Ann", "Sadley", "Ling", "Gongaware", "Schmidt", "Endsley",
+            "Groah", "Flanders", "Reichard", "Lowstetter", "Sandblom", "Griffis", "Basmanoff", "Coveney", "Hawker",
+            "Archibald", "Hutton", "Barnes", "Diegel", "Raybould", "Focell", "Breitenstein", "Murray", "Chauvin",
+            "Busk", "Pheleps", "Teagarden", "Northey", "Baumgartner", "Fleming", "Harris", "Parkinson", "Carpenter",
+            "Whirlow", "Bonner", "Wortman", "Rogers", "Scott", "Lowe", "Mckee", "Huston", "Bullard", "Throckmorton",
+            "Rummel", "Mathews", "Dull", "Saline", "Tue", "Woolery", "Lalty", "Schrader", "Ramsey", "Eisenmann",
+            "Philbrick", "Sybilla", "Wallace", "Fonblanque", "Paul", "Orbell", "Higgens", "Casteel", "Franks",
+            "Demuth", "Eisenman", "Hay", "Robinson", "Fischer", "Hincken", "Wylie", "Leichter", "Bousum",
+            "Littlefield", "Mcdonald", "Greif", "Rhodes", "Wall", "Steele", "Baldwin", "Smith", "Stewart", "Schere",
+            "Mary", "Aultman", "Emrick", "Guess", "Mitchell", "Painter", "Aft", "Hasely", "Weldi", "Loewentsein",
+            "Poorbaugh", "Kepple", "Noton", "Judge", "Jackson", "Style", "Adcock", "Diller", "Marriman", "Johnston",
+            "Children", "Monahan", "Ehret", "Shaw", "Congdon", "Pinney", "Millard", "Crissman", "Tanner", "Rose",
+            "Knisely", "Cypret", "Sommer", "Poehl", "Hardie", "Bender", "Overholt", "Gottwine", "Beach", "Leslie",
+            "Trevithick", "Langston", "Magor", "Shotts", "Howe", "Hunter", "Cross", "Kistler", "Dealtry", "Christner",
+            "Pennington", "Thorley", "Eckhardstein", "Van", "Stroh", "Stough", "Stall", "Beedell", "Shea", "Garland",
+            "Mays", "Pritchard", "Frankenberger", "Rowley", "Lane", "Baum", "Alliman", "Park", "Jardine", "Butler",
+            "Cherry", "Kooser", "Baxter", "Billimek", "Downing", "Hurst", "Wood", "Baird", "Watkins", "Edwards",
+            "Kemerer", "Harding", "Owens", "Eiford", "Keener", "Garneis", "Fiscina", "Mang", "Draudy", "Mills",
+            "Gibson", "Reese", "Todd", "Ramos", "Levett", "Wilks", "Ward", "Mosser", "Dunlap", "Kifer", "Christopher",
+            "Ashbaugh", "Wynter", "Rawls", "Cribbs", "Haynes", "Thigpen", "Schreckengost", "Bishop", "Linton",
+            "Chapman", "James", "Jerome", "Hook", "Omara", "Houston", "Maclagan", "Sandys", "Pickering", "Blois",
+            "Dickson", "Kemble", "Duncan", "Woodward", "Southern", "Henley", "Treeby", "Cram", "Elsas", "Driggers",
+            "Warrick", "Overstreet", "Hindman", "Buck", "Sulyard", "Wentzel", "Swink", "Butt", "Schaeffer",
+            "Hoffhants", "Bould", "Willcox", "Lotherington", "Bagley", "Graff", "White", "Wheeler", "Sloan",
+            "Rodacker", "Hanford", "Jowers", "Kunkle", "Cass", "Powers", "Gilman", "Mcmichaels", "Hobbs", "Herndon",
+            "Prescott", "Smail", "Mcdonald", "Biery", "Orner", "Richards", "Mueller", "Isaman", "Bruxner", "Goodman",
+            "Barth", "Turzanski", "Vorrasi", "Stainforth", "Nehling", "Rahl", "Erschoff", "Greene", "Mckinnon",
+            "Reade", "Smith", "Pery", "Roose", "Greenwood", "Weisgarber", "Curry", "Holts", "Zadovsky", "Parrish",
+            "Putnam", "Munson", "Mcindoe", "Nickolson", "Brooks", "Bollinger", "Stroble", "Siegrist", "Fulton",
+            "Tomey", "Zoucks", "Roberts", "Otis", "Clarke", "Easter", "Johnson", "Fylbrigg", "Taylor", "Swartzbaugh",
+            "Weinstein", "Gadow", "Sayre", "Marcotte", "Wise", "Atweeke", "Mcfall", "Napier", "Eisenhart", "Canham",
+            "Sealis", "Baughman", "Gertraht", "Losey", "Laurence", "Eva", "Pershing", "Kern", "Pirl", "Rega",
+            "Sanborn", "Kanaga", "Sanders", "Anderson", "Dickinson", "Osteen", "Gettemy", "Crom", "Snyder", "Reed",
+            "Laurenzi", "Riggle", "Tillson", "Fowler", "Raub", "Jenner", "Koepple", "Soames", "Goldvogel", "Dimsdale",
+            "Zimmer", "Giesen", "Baker", "Beail", "Mortland", "Bard", "Sanner", "Knopsnider", "Jenkins", "Bailey",
+            "Werner", "Barrett", "Faust", "Agg", "Tomlinson", "Williams", "Little", "Greenawalt", "Wells", "Wilkins",
+            "Gisiko", "Bauerle", "Harrold", "Prechtl", "Polson", "Faast", "Winton", "Garneys", "Peters", "Potter",
+            "Porter", "Tennant", "Eve", "Dugger", "Jones", "Burch", "Cowper", "Whittier" };
+
+    public static String[] firstNames = { "Albert", "Jacquelin", "Dona", "Alia", "Mayme", "Genoveva", "Emma", "Lena",
+            "Melody", "Vilma", "Katelyn", "Jeremy", "Coral", "Leann", "Lita", "Gilda", "Kayla", "Alvina", "Maranda",
+            "Verlie", "Khadijah", "Karey", "Patrice", "Kallie", "Corey", "Mollie", "Daisy", "Melanie", "Sarita",
+            "Nichole", "Pricilla", "Terresa", "Berneice", "Arianne", "Brianne", "Lavinia", "Ulrike", "Lesha", "Adell",
+            "Ardelle", "Marisha", "Laquita", "Karyl", "Maryjane", "Kendall", "Isobel", "Raeann", "Heike", "Barbera",
+            "Norman", "Yasmine", "Nevada", "Mariam", "Edith", "Eugena", "Lovie", "Maren", "Bennie", "Lennie", "Tamera",
+            "Crystal", "Randi", "Anamaria", "Chantal", "Jesenia", "Avis", "Shela", "Randy", "Laurena", "Sharron",
+            "Christiane", "Lorie", "Mario", "Elizabeth", "Reina", "Adria", "Lakisha", "Brittni", "Azzie", "Dori",
+            "Shaneka", "Asuncion", "Katheryn", "Laurice", "Sharita", "Krystal", "Reva", "Inger", "Alpha", "Makeda",
+            "Anabel", "Loni", "Tiara", "Meda", "Latashia", "Leola", "Chin", "Daisey", "Ivory", "Amalia", "Logan",
+            "Tyler", "Kyong", "Carolann", "Maryetta", "Eufemia", "Anya", "Doreatha", "Lorna", "Rutha", "Ehtel",
+            "Debbie", "Chassidy", "Sang", "Christa", "Lottie", "Chun", "Karine", "Peggie", "Amina", "Melany", "Alayna",
+            "Scott", "Romana", "Naomi", "Christiana", "Salena", "Taunya", "Mitsue", "Regina", "Chelsie", "Charity",
+            "Dacia", "Aletha", "Latosha", "Lia", "Tamica", "Chery", "Bianca", "Shu", "Georgianne", "Myriam", "Austin",
+            "Wan", "Mallory", "Jana", "Georgie", "Jenell", "Kori", "Vicki", "Delfina", "June", "Mellisa", "Catherina",
+            "Claudie", "Tynisha", "Dayle", "Enriqueta", "Belen", "Pia", "Sarai", "Rosy", "Renay", "Kacie", "Frieda",
+            "Cayla", "Elissa", "Claribel", "Sabina", "Mackenzie", "Raina", "Cira", "Mitzie", "Aubrey", "Serafina",
+            "Maria", "Katharine", "Esperanza", "Sung", "Daria", "Billye", "Stefanie", "Kasha", "Holly", "Suzanne",
+            "Inga", "Flora", "Andria", "Genevie", "Eladia", "Janet", "Erline", "Renna", "Georgeanna", "Delorse",
+            "Elnora", "Rudy", "Rima", "Leanora", "Letisha", "Love", "Alverta", "Pinkie", "Domonique", "Jeannie",
+            "Jose", "Jacqueline", "Tara", "Lily", "Erna", "Tennille", "Galina", "Tamala", "Kirby", "Nichelle",
+            "Myesha", "Farah", "Santa", "Ludie", "Kenia", "Yee", "Micheline", "Maryann", "Elaina", "Ethelyn",
+            "Emmaline", "Shanell", "Marina", "Nila", "Alane", "Shakira", "Dorris", "Belinda", "Elois", "Barbie",
+            "Carita", "Gisela", "Lura", "Fransisca", "Helga", "Peg", "Leonarda", "Earlie", "Deetta", "Jacquetta",
+            "Blossom", "Kayleigh", "Deloras", "Keshia", "Christinia", "Dulce", "Bernie", "Sheba", "Lashanda", "Tula",
+            "Claretta", "Kary", "Jeanette", "Lupita", "Lenora", "Hisako", "Sherise", "Glynda", "Adela", "Chia",
+            "Sudie", "Mindy", "Caroyln", "Lindsey", "Xiomara", "Mercedes", "Onie", "Loan", "Alexis", "Tommie",
+            "Donette", "Monica", "Soo", "Camellia", "Lavera", "Valery", "Ariana", "Sophia", "Loris", "Ginette",
+            "Marielle", "Tari", "Julissa", "Alesia", "Suzanna", "Emelda", "Erin", "Ladawn", "Sherilyn", "Candice",
+            "Nereida", "Fairy", "Carl", "Joel", "Marilee", "Gracia", "Cordie", "So", "Shanita", "Drew", "Cassie",
+            "Sherie", "Marget", "Norma", "Delois", "Debera", "Chanelle", "Catarina", "Aracely", "Carlene", "Tricia",
+            "Aleen", "Katharina", "Marguerita", "Guadalupe", "Margorie", "Mandie", "Kathe", "Chong", "Sage", "Faith",
+            "Maryrose", "Stephany", "Ivy", "Pauline", "Susie", "Cristen", "Jenifer", "Annette", "Debi", "Karmen",
+            "Luci", "Shayla", "Hope", "Ocie", "Sharie", "Tami", "Breana", "Kerry", "Rubye", "Lashay", "Sondra",
+            "Katrice", "Brunilda", "Cortney", "Yan", "Zenobia", "Penni", "Addie", "Lavona", "Noel", "Anika",
+            "Herlinda", "Valencia", "Bunny", "Tory", "Victoria", "Carrie", "Mikaela", "Wilhelmina", "Chung",
+            "Hortencia", "Gerda", "Wen", "Ilana", "Sibyl", "Candida", "Victorina", "Chantell", "Casie", "Emeline",
+            "Dominica", "Cecila", "Delora", "Miesha", "Nova", "Sally", "Ronald", "Charlette", "Francisca", "Mina",
+            "Jenna", "Loraine", "Felisa", "Lulu", "Page", "Lyda", "Babara", "Flor", "Walter", "Chan", "Sherika",
+            "Kala", "Luna", "Vada", "Syreeta", "Slyvia", "Karin", "Renata", "Robbi", "Glenda", "Delsie", "Lizzie",
+            "Genia", "Caitlin", "Bebe", "Cory", "Sam", "Leslee", "Elva", "Caren", "Kasie", "Leticia", "Shannan",
+            "Vickey", "Sandie", "Kyle", "Chang", "Terrilyn", "Sandra", "Elida", "Marketta", "Elsy", "Tu", "Carman",
+            "Ashlie", "Vernia", "Albertine", "Vivian", "Elba", "Bong", "Margy", "Janetta", "Xiao", "Teofila", "Danyel",
+            "Nickole", "Aleisha", "Tera", "Cleotilde", "Dara", "Paulita", "Isela", "Maricela", "Rozella", "Marivel",
+            "Aurora", "Melissa", "Carylon", "Delinda", "Marvella", "Candelaria", "Deidre", "Tawanna", "Myrtie",
+            "Milagro", "Emilie", "Coretta", "Ivette", "Suzann", "Ammie", "Lucina", "Lory", "Tena", "Eleanor",
+            "Cherlyn", "Tiana", "Brianna", "Myra", "Flo", "Carisa", "Kandi", "Erlinda", "Jacqulyn", "Fermina", "Riva",
+            "Palmira", "Lindsay", "Annmarie", "Tamiko", "Carline", "Amelia", "Quiana", "Lashawna", "Veola", "Belva",
+            "Marsha", "Verlene", "Alex", "Leisha", "Camila", "Mirtha", "Melva", "Lina", "Arla", "Cythia", "Towanda",
+            "Aracelis", "Tasia", "Aurore", "Trinity", "Bernadine", "Farrah", "Deneen", "Ines", "Betty", "Lorretta",
+            "Dorethea", "Hertha", "Rochelle", "Juli", "Shenika", "Yung", "Lavon", "Deeanna", "Nakia", "Lynnette",
+            "Dinorah", "Nery", "Elene", "Carolee", "Mira", "Franchesca", "Lavonda", "Leida", "Paulette", "Dorine",
+            "Allegra", "Keva", "Jeffrey", "Bernardina", "Maryln", "Yoko", "Faviola", "Jayne", "Lucilla", "Charita",
+            "Ewa", "Ella", "Maggie", "Ivey", "Bettie", "Jerri", "Marni", "Bibi", "Sabrina", "Sarah", "Marleen",
+            "Katherin", "Remona", "Jamika", "Antonina", "Oliva", "Lajuana", "Fonda", "Sigrid", "Yael", "Billi",
+            "Verona", "Arminda", "Mirna", "Tesha", "Katheleen", "Bonita", "Kamilah", "Patrica", "Julio", "Shaina",
+            "Mellie", "Denyse", "Deandrea", "Alena", "Meg", "Kizzie", "Krissy", "Karly", "Alleen", "Yahaira", "Lucie",
+            "Karena", "Elaine", "Eloise", "Buena", "Marianela", "Renee", "Nan", "Carolynn", "Windy", "Avril", "Jane",
+            "Vida", "Thea", "Marvel", "Rosaline", "Tifany", "Robena", "Azucena", "Carlota", "Mindi", "Andera", "Jenny",
+            "Courtney", "Lyndsey", "Willette", "Kristie", "Shaniqua", "Tabatha", "Ngoc", "Una", "Marlena", "Louetta",
+            "Vernie", "Brandy", "Jacquelyne", "Jenelle", "Elna", "Erminia", "Ida", "Audie", "Louis", "Marisol",
+            "Shawana", "Harriette", "Karol", "Kitty", "Esmeralda", "Vivienne", "Eloisa", "Iris", "Jeanice", "Cammie",
+            "Jacinda", "Shena", "Floy", "Theda", "Lourdes", "Jayna", "Marg", "Kati", "Tanna", "Rosalyn", "Maxima",
+            "Soon", "Angelika", "Shonna", "Merle", "Kassandra", "Deedee", "Heidi", "Marti", "Renae", "Arleen",
+            "Alfredia", "Jewell", "Carley", "Pennie", "Corina", "Tonisha", "Natividad", "Lilliana", "Darcie", "Shawna",
+            "Angel", "Piedad", "Josefa", "Rebbeca", "Natacha", "Nenita", "Petrina", "Carmon", "Chasidy", "Temika",
+            "Dennise", "Renetta", "Augusta", "Shirlee", "Valeri", "Casimira", "Janay", "Berniece", "Deborah", "Yaeko",
+            "Mimi", "Digna", "Irish", "Cher", "Yong", "Lucila", "Jimmie", "Junko", "Lezlie", "Waneta", "Sandee",
+            "Marquita", "Eura", "Freeda", "Annabell", "Laree", "Jaye", "Wendy", "Toshia", "Kylee", "Aleta", "Emiko",
+            "Clorinda", "Sixta", "Audrea", "Juanita", "Birdie", "Reita", "Latanya", "Nia", "Leora", "Laurine",
+            "Krysten", "Jerrie", "Chantel", "Ira", "Sena", "Andre", "Jann", "Marla", "Precious", "Katy", "Gabrielle",
+            "Yvette", "Brook", "Shirlene", "Eldora", "Laura", "Milda", "Euna", "Jettie", "Debora", "Lise", "Edythe",
+            "Leandra", "Shandi", "Araceli", "Johanne", "Nieves", "Denese", "Carmelita", "Nohemi", "Annice", "Natalie",
+            "Yolande", "Jeffie", "Vashti", "Vickie", "Obdulia", "Youlanda", "Lupe", "Tomoko", "Monserrate", "Domitila",
+            "Etsuko", "Adrienne", "Lakesha", "Melissia", "Odessa", "Meagan", "Veronika", "Jolyn", "Isabelle", "Leah",
+            "Rhiannon", "Gianna", "Audra", "Sommer", "Renate", "Perla", "Thao", "Myong", "Lavette", "Mark", "Emilia",
+            "Ariane", "Karl", "Dorie", "Jacquie", "Mia", "Malka", "Shenita", "Tashina", "Christine", "Cherri", "Roni",
+            "Fran", "Mildred", "Sara", "Clarissa", "Fredia", "Elease", "Samuel", "Earlene", "Vernita", "Mae", "Concha",
+            "Renea", "Tamekia", "Hye", "Ingeborg", "Tessa", "Kelly", "Kristin", "Tam", "Sacha", "Kanisha", "Jillian",
+            "Tiffanie", "Ashlee", "Madelyn", "Donya", "Clementine", "Mickie", "My", "Zena", "Terrie", "Samatha",
+            "Gertie", "Tarra", "Natalia", "Sharlene", "Evie", "Shalon", "Rosalee", "Numbers", "Jodi", "Hattie",
+            "Naoma", "Valene", "Whitley", "Claude", "Alline", "Jeanne", "Camie", "Maragret", "Viola", "Kris", "Marlo",
+            "Arcelia", "Shari", "Jalisa", "Corrie", "Eleonor", "Angelyn", "Merry", "Lauren", "Melita", "Gita",
+            "Elenor", "Aurelia", "Janae", "Lyndia", "Margeret", "Shawanda", "Rolande", "Shirl", "Madeleine", "Celinda",
+            "Jaleesa", "Shemika", "Joye", "Tisa", "Trudie", "Kathrine", "Clarita", "Dinah", "Georgia", "Antoinette",
+            "Janis", "Suzette", "Sherri", "Herta", "Arie", "Hedy", "Cassi", "Audrie", "Caryl", "Jazmine", "Jessica",
+            "Beverly", "Elizbeth", "Marylee", "Londa", "Fredericka", "Argelia", "Nana", "Donnette", "Damaris",
+            "Hailey", "Jamee", "Kathlene", "Glayds", "Lydia", "Apryl", "Verla", "Adam", "Concepcion", "Zelda",
+            "Shonta", "Vernice", "Detra", "Meghann", "Sherley", "Sheri", "Kiyoko", "Margarita", "Adaline", "Mariela",
+            "Velda", "Ailene", "Juliane", "Aiko", "Edyth", "Cecelia", "Shavon", "Florance", "Madeline", "Rheba",
+            "Deann", "Ignacia", "Odelia", "Heide", "Mica", "Jennette", "Maricruz", "Ouida", "Darcy", "Laure",
+            "Justina", "Amada", "Laine", "Cruz", "Sunny", "Francene", "Roxanna", "Nam", "Nancie", "Deanna", "Letty",
+            "Britni", "Kazuko", "Lacresha", "Simon", "Caleb", "Milton", "Colton", "Travis", "Miles", "Jonathan",
+            "Logan", "Rolf", "Emilio", "Roberto", "Marcus", "Tim", "Delmar", "Devon", "Kurt", "Edward", "Jeffrey",
+            "Elvis", "Alfonso", "Blair", "Wm", "Sheldon", "Leonel", "Michal", "Federico", "Jacques", "Leslie",
+            "Augustine", "Hugh", "Brant", "Hong", "Sal", "Modesto", "Curtis", "Jefferey", "Adam", "John", "Glenn",
+            "Vance", "Alejandro", "Refugio", "Lucio", "Demarcus", "Chang", "Huey", "Neville", "Preston", "Bert",
+            "Abram", "Foster", "Jamison", "Kirby", "Erich", "Manual", "Dustin", "Derrick", "Donnie", "Jospeh", "Chris",
+            "Josue", "Stevie", "Russ", "Stanley", "Nicolas", "Samuel", "Waldo", "Jake", "Max", "Ernest", "Reinaldo",
+            "Rene", "Gale", "Morris", "Nathan", "Maximo", "Courtney", "Theodore", "Octavio", "Otha", "Delmer",
+            "Graham", "Dean", "Lowell", "Myles", "Colby", "Boyd", "Adolph", "Jarrod", "Nick", "Mark", "Clinton", "Kim",
+            "Sonny", "Dalton", "Tyler", "Jody", "Orville", "Luther", "Rubin", "Hollis", "Rashad", "Barton", "Vicente",
+            "Ted", "Rick", "Carmine", "Clifton", "Gayle", "Christopher", "Jessie", "Bradley", "Clay", "Theo", "Josh",
+            "Mitchell", "Boyce", "Chung", "Eugenio", "August", "Norbert", "Sammie", "Jerry", "Adan", "Edmundo",
+            "Homer", "Hilton", "Tod", "Kirk", "Emmett", "Milan", "Quincy", "Jewell", "Herb", "Steve", "Carmen",
+            "Bobby", "Odis", "Daron", "Jeremy", "Carl", "Hunter", "Tuan", "Thurman", "Asa", "Brenton", "Shane",
+            "Donny", "Andreas", "Teddy", "Dario", "Cyril", "Hoyt", "Teodoro", "Vincenzo", "Hilario", "Daren",
+            "Agustin", "Marquis", "Ezekiel", "Brendan", "Johnson", "Alden", "Richie", "Granville", "Chad", "Joseph",
+            "Lamont", "Jordon", "Gilberto", "Chong", "Rosendo", "Eddy", "Rob", "Dewitt", "Andre", "Titus", "Russell",
+            "Rigoberto", "Dick", "Garland", "Gabriel", "Hank", "Darius", "Ignacio", "Lazaro", "Johnie", "Mauro",
+            "Edmund", "Trent", "Harris", "Osvaldo", "Marvin", "Judson", "Rodney", "Randall", "Renato", "Richard",
+            "Denny", "Jon", "Doyle", "Cristopher", "Wilson", "Christian", "Jamie", "Roland", "Ken", "Tad", "Romeo",
+            "Seth", "Quinton", "Byron", "Ruben", "Darrel", "Deandre", "Broderick", "Harold", "Ty", "Monroe", "Landon",
+            "Mohammed", "Angel", "Arlen", "Elias", "Andres", "Carlton", "Numbers", "Tony", "Thaddeus", "Issac",
+            "Elmer", "Antoine", "Ned", "Fermin", "Grover", "Benito", "Abdul", "Cortez", "Eric", "Maxwell", "Coy",
+            "Gavin", "Rich", "Andy", "Del", "Giovanni", "Major", "Efren", "Horacio", "Joaquin", "Charles", "Noah",
+            "Deon", "Pasquale", "Reed", "Fausto", "Jermaine", "Irvin", "Ray", "Tobias", "Carter", "Yong", "Jorge",
+            "Brent", "Daniel", "Zane", "Walker", "Thad", "Shaun", "Jaime", "Mckinley", "Bradford", "Nathanial",
+            "Jerald", "Aubrey", "Virgil", "Abel", "Philip", "Chester", "Chadwick", "Dominick", "Britt", "Emmitt",
+            "Ferdinand", "Julian", "Reid", "Santos", "Dwain", "Morgan", "James", "Marion", "Micheal", "Eddie", "Brett",
+            "Stacy", "Kerry", "Dale", "Nicholas", "Darrick", "Freeman", "Scott", "Newton", "Sherman", "Felton",
+            "Cedrick", "Winfred", "Brad", "Fredric", "Dewayne", "Virgilio", "Reggie", "Edgar", "Heriberto", "Shad",
+            "Timmy", "Javier", "Nestor", "Royal", "Lynn", "Irwin", "Ismael", "Jonas", "Wiley", "Austin", "Kieth",
+            "Gonzalo", "Paris", "Earnest", "Arron", "Jarred", "Todd", "Erik", "Maria", "Chauncey", "Neil", "Conrad",
+            "Maurice", "Roosevelt", "Jacob", "Sydney", "Lee", "Basil", "Louis", "Rodolfo", "Rodger", "Roman", "Corey",
+            "Ambrose", "Cristobal", "Sylvester", "Benton", "Franklin", "Marcelo", "Guillermo", "Toby", "Jeramy",
+            "Donn", "Danny", "Dwight", "Clifford", "Valentine", "Matt", "Jules", "Kareem", "Ronny", "Lonny", "Son",
+            "Leopoldo", "Dannie", "Gregg", "Dillon", "Orlando", "Weston", "Kermit", "Damian", "Abraham", "Walton",
+            "Adrian", "Rudolf", "Will", "Les", "Norberto", "Fred", "Tyrone", "Ariel", "Terry", "Emmanuel", "Anderson",
+            "Elton", "Otis", "Derek", "Frankie", "Gino", "Lavern", "Jarod", "Kenny", "Dane", "Keenan", "Bryant",
+            "Eusebio", "Dorian", "Ali", "Lucas", "Wilford", "Jeremiah", "Warner", "Woodrow", "Galen", "Bob",
+            "Johnathon", "Amado", "Michel", "Harry", "Zachery", "Taylor", "Booker", "Hershel", "Mohammad", "Darrell",
+            "Kyle", "Stuart", "Marlin", "Hyman", "Jeffery", "Sidney", "Merrill", "Roy", "Garrett", "Porter", "Kenton",
+            "Giuseppe", "Terrance", "Trey", "Felix", "Buster", "Von", "Jackie", "Linwood", "Darron", "Francisco",
+            "Bernie", "Diego", "Brendon", "Cody", "Marco", "Ahmed", "Antonio", "Vince", "Brooks", "Kendrick", "Ross",
+            "Mohamed", "Jim", "Benny", "Gerald", "Pablo", "Charlie", "Antony", "Werner", "Hipolito", "Minh", "Mel",
+            "Derick", "Armand", "Fidel", "Lewis", "Donnell", "Desmond", "Vaughn", "Guadalupe", "Keneth", "Rodrick",
+            "Spencer", "Chas", "Gus", "Harlan", "Wes", "Carmelo", "Jefferson", "Gerard", "Jarvis", "Haywood", "Hayden",
+            "Sergio", "Gene", "Edgardo", "Colin", "Horace", "Dominic", "Aldo", "Adolfo", "Juan", "Man", "Lenard",
+            "Clement", "Everett", "Hal", "Bryon", "Mason", "Emerson", "Earle", "Laurence", "Columbus", "Lamar",
+            "Douglas", "Ian", "Fredrick", "Marc", "Loren", "Wallace", "Randell", "Noble", "Ricardo", "Rory", "Lindsey",
+            "Boris", "Bill", "Carlos", "Domingo", "Grant", "Craig", "Ezra", "Matthew", "Van", "Rudy", "Danial",
+            "Brock", "Maynard", "Vincent", "Cole", "Damion", "Ellsworth", "Marcel", "Markus", "Rueben", "Tanner",
+            "Reyes", "Hung", "Kennith", "Lindsay", "Howard", "Ralph", "Jed", "Monte", "Garfield", "Avery", "Bernardo",
+            "Malcolm", "Sterling", "Ezequiel", "Kristofer", "Luciano", "Casey", "Rosario", "Ellis", "Quintin",
+            "Trevor", "Miquel", "Jordan", "Arthur", "Carson", "Tyron", "Grady", "Walter", "Jonathon", "Ricky",
+            "Bennie", "Terrence", "Dion", "Dusty", "Roderick", "Isaac", "Rodrigo", "Harrison", "Zack", "Dee", "Devin",
+            "Rey", "Ulysses", "Clint", "Greg", "Dino", "Frances", "Wade", "Franklyn", "Jude", "Bradly", "Salvador",
+            "Rocky", "Weldon", "Lloyd", "Milford", "Clarence", "Alec", "Allan", "Bobbie", "Oswaldo", "Wilfred",
+            "Raleigh", "Shelby", "Willy", "Alphonso", "Arnoldo", "Robbie", "Truman", "Nicky", "Quinn", "Damien",
+            "Lacy", "Marcos", "Parker", "Burt", "Carroll", "Denver", "Buck", "Dong", "Normand", "Billie", "Edwin",
+            "Troy", "Arden", "Rusty", "Tommy", "Kenneth", "Leo", "Claud", "Joel", "Kendall", "Dante", "Milo", "Cruz",
+            "Lucien", "Ramon", "Jarrett", "Scottie", "Deshawn", "Ronnie", "Pete", "Alonzo", "Whitney", "Stefan",
+            "Sebastian", "Edmond", "Enrique", "Branden", "Leonard", "Loyd", "Olin", "Ron", "Rhett", "Frederic",
+            "Orval", "Tyrell", "Gail", "Eli", "Antonia", "Malcom", "Sandy", "Stacey", "Nickolas", "Hosea", "Santo",
+            "Oscar", "Fletcher", "Dave", "Patrick", "Dewey", "Bo", "Vito", "Blaine", "Randy", "Robin", "Winston",
+            "Sammy", "Edwardo", "Manuel", "Valentin", "Stanford", "Filiberto", "Buddy", "Zachariah", "Johnnie",
+            "Elbert", "Paul", "Isreal", "Jerrold", "Leif", "Owen", "Sung", "Junior", "Raphael", "Josef", "Donte",
+            "Allen", "Florencio", "Raymond", "Lauren", "Collin", "Eliseo", "Bruno", "Martin", "Lyndon", "Kurtis",
+            "Salvatore", "Erwin", "Michael", "Sean", "Davis", "Alberto", "King", "Rolland", "Joe", "Tory", "Chase",
+            "Dallas", "Vernon", "Beau", "Terrell", "Reynaldo", "Monty", "Jame", "Dirk", "Florentino", "Reuben", "Saul",
+            "Emory", "Esteban", "Michale", "Claudio", "Jacinto", "Kelley", "Levi", "Andrea", "Lanny", "Wendell",
+            "Elwood", "Joan", "Felipe", "Palmer", "Elmo", "Lawrence", "Hubert", "Rudolph", "Duane", "Cordell",
+            "Everette", "Mack", "Alan", "Efrain", "Trenton", "Bryan", "Tom", "Wilmer", "Clyde", "Chance", "Lou",
+            "Brain", "Justin", "Phil", "Jerrod", "George", "Kris", "Cyrus", "Emery", "Rickey", "Lincoln", "Renaldo",
+            "Mathew", "Luke", "Dwayne", "Alexis", "Jackson", "Gil", "Marty", "Burton", "Emil", "Glen", "Willian",
+            "Clemente", "Keven", "Barney", "Odell", "Reginald", "Aurelio", "Damon", "Ward", "Gustavo", "Harley",
+            "Peter", "Anibal", "Arlie", "Nigel", "Oren", "Zachary", "Scot", "Bud", "Wilbert", "Bart", "Josiah",
+            "Marlon", "Eldon", "Darryl", "Roger", "Anthony", "Omer", "Francis", "Patricia", "Moises", "Chuck",
+            "Waylon", "Hector", "Jamaal", "Cesar", "Julius", "Rex", "Norris", "Ollie", "Isaias", "Quentin", "Graig",
+            "Lyle", "Jeffry", "Karl", "Lester", "Danilo", "Mike", "Dylan", "Carlo", "Ryan", "Leon", "Percy", "Lucius",
+            "Jamel", "Lesley", "Joey", "Cornelius", "Rico", "Arnulfo", "Chet", "Margarito", "Ernie", "Nathanael",
+            "Amos", "Cleveland", "Luigi", "Alfonzo", "Phillip", "Clair", "Elroy", "Alva", "Hans", "Shon", "Gary",
+            "Jesus", "Cary", "Silas", "Keith", "Israel", "Willard", "Randolph", "Dan", "Adalberto", "Claude",
+            "Delbert", "Garry", "Mary", "Larry", "Riley", "Robt", "Darwin", "Barrett", "Steven", "Kelly", "Herschel",
+            "Darnell", "Scotty", "Armando", "Miguel", "Lawerence", "Wesley", "Garth", "Carol", "Micah", "Alvin",
+            "Billy", "Earl", "Pat", "Brady", "Cory", "Carey", "Bernard", "Jayson", "Nathaniel", "Gaylord", "Archie",
+            "Dorsey", "Erasmo", "Angelo", "Elisha", "Long", "Augustus", "Hobert", "Drew", "Stan", "Sherwood",
+            "Lorenzo", "Forrest", "Shawn", "Leigh", "Hiram", "Leonardo", "Gerry", "Myron", "Hugo", "Alvaro", "Leland",
+            "Genaro", "Jamey", "Stewart", "Elden", "Irving", "Olen", "Antone", "Freddy", "Lupe", "Joshua", "Gregory",
+            "Andrew", "Sang", "Wilbur", "Gerardo", "Merlin", "Williams", "Johnny", "Alex", "Tommie", "Jimmy",
+            "Donovan", "Dexter", "Gaston", "Tracy", "Jeff", "Stephen", "Berry", "Anton", "Darell", "Fritz", "Willis",
+            "Noel", "Mariano", "Crawford", "Zoey", "Alex", "Brianna", "Carlie", "Lloyd", "Cal", "Astor", "Randolf",
+            "Magdalene", "Trevelyan", "Terance", "Roy", "Kermit", "Harriett", "Crystal", "Laurinda", "Kiersten",
+            "Phyllida", "Liz", "Bettie", "Rena", "Colten", "Berenice", "Sindy", "Wilma", "Amos", "Candi", "Ritchie",
+            "Dirk", "Kathlyn", "Callista", "Anona", "Flossie", "Sterling", "Calista", "Regan", "Erica", "Jeana",
+            "Keaton", "York", "Nolan", "Daniel", "Benton", "Tommie", "Serenity", "Deanna", "Chas", "Heron", "Marlyn",
+            "Xylia", "Tristin", "Lyndon", "Andriana", "Madelaine", "Maddison", "Leila", "Chantelle", "Audrey",
+            "Connor", "Daley", "Tracee", "Tilda", "Eliot", "Merle", "Linwood", "Kathryn", "Silas", "Alvina",
+            "Phinehas", "Janis", "Alvena", "Zubin", "Gwendolen", "Caitlyn", "Bertram", "Hailee", "Idelle", "Homer",
+            "Jannah", "Delbert", "Rhianna", "Cy", "Jefferson", "Wayland", "Nona", "Tempest", "Reed", "Jenifer",
+            "Ellery", "Nicolina", "Aldous", "Prince", "Lexia", "Vinnie", "Doug", "Alberic", "Kayleen", "Woody",
+            "Rosanne", "Ysabel", "Skyler", "Twyla", "Geordie", "Leta", "Clive", "Aaron", "Scottie", "Celeste", "Chuck",
+            "Erle", "Lallie", "Jaycob", "Ray", "Carrie", "Laurita", "Noreen", "Meaghan", "Ulysses", "Andy", "Drogo",
+            "Dina", "Yasmin", "Mya", "Luvenia", "Urban", "Jacob", "Laetitia", "Sherry", "Love", "Michaela", "Deonne",
+            "Summer", "Brendon", "Sheena", "Mason", "Jayson", "Linden", "Salal", "Darrell", "Diana", "Hudson",
+            "Lennon", "Isador", "Charley", "April", "Ralph", "James", "Mina", "Jolyon", "Laurine", "Monna", "Carita",
+            "Munro", "Elsdon", "Everette", "Radclyffe", "Darrin", "Herbert", "Gawain", "Sheree", "Trudy", "Emmaline",
+            "Kassandra", "Rebecca", "Basil", "Jen", "Don", "Osborne", "Lilith", "Hannah", "Fox", "Rupert", "Paulene",
+            "Darius", "Wally", "Baptist", "Sapphire", "Tia", "Sondra", "Kylee", "Ashton", "Jepson", "Joetta", "Val",
+            "Adela", "Zacharias", "Zola", "Marmaduke", "Shannah", "Posie", "Oralie", "Brittany", "Ernesta", "Raymund",
+            "Denzil", "Daren", "Roosevelt", "Nelson", "Fortune", "Mariel", "Nick", "Jaden", "Upton", "Oz", "Margaux",
+            "Precious", "Albert", "Bridger", "Jimmy", "Nicola", "Rosalynne", "Keith", "Walt", "Della", "Joanna",
+            "Xenia", "Esmeralda", "Major", "Simon", "Rexana", "Stacy", "Calanthe", "Sherley", "Kaitlyn", "Graham",
+            "Ramsey", "Abbey", "Madlyn", "Kelvin", "Bill", "Rue", "Monica", "Caileigh", "Laraine", "Booker", "Jayna",
+            "Greta", "Jervis", "Sherman", "Kendrick", "Tommy", "Iris", "Geffrey", "Kaelea", "Kerr", "Garrick", "Jep",
+            "Audley", "Nic", "Bronte", "Beulah", "Patricia", "Jewell", "Deidra", "Cory", "Everett", "Harper",
+            "Charity", "Godfrey", "Jaime", "Sinclair", "Talbot", "Dayna", "Cooper", "Rosaline", "Jennie", "Eileen",
+            "Latanya", "Corinna", "Roxie", "Caesar", "Charles", "Pollie", "Lindsey", "Sorrel", "Dwight", "Jocelyn",
+            "Weston", "Shyla", "Valorie", "Bessie", "Josh", "Lessie", "Dayton", "Kathi", "Chasity", "Wilton", "Adam",
+            "William", "Ash", "Angela", "Ivor", "Ria", "Jazmine", "Hailey", "Jo", "Silvestra", "Ernie", "Clifford",
+            "Levi", "Matilda", "Quincey", "Camilla", "Delicia", "Phemie", "Laurena", "Bambi", "Lourdes", "Royston",
+            "Chastity", "Lynwood", "Elle", "Brenda", "Phoebe", "Timothy", "Raschelle", "Lilly", "Burt", "Rina",
+            "Rodney", "Maris", "Jaron", "Wilf", "Harlan", "Audra", "Vincent", "Elwyn", "Drew", "Wynter", "Ora",
+            "Lissa", "Virgil", "Xavier", "Chad", "Ollie", "Leyton", "Karolyn", "Skye", "Roni", "Gladys", "Dinah",
+            "Penny", "August", "Osmund", "Whitaker", "Brande", "Cornell", "Phil", "Zara", "Kilie", "Gavin", "Coty",
+            "Randy", "Teri", "Keira", "Pru", "Clemency", "Kelcey", "Nevil", "Poppy", "Gareth", "Christabel", "Bastian",
+            "Wynonna", "Roselyn", "Goddard", "Collin", "Trace", "Neal", "Effie", "Denys", "Virginia", "Richard",
+            "Isiah", "Harrietta", "Gaylord", "Diamond", "Trudi", "Elaine", "Jemmy", "Gage", "Annabel", "Quincy", "Syd",
+            "Marianna", "Philomena", "Aubree", "Kathie", "Jacki", "Kelley", "Bess", "Cecil", "Maryvonne", "Kassidy",
+            "Anselm", "Dona", "Darby", "Jamison", "Daryl", "Darell", "Teal", "Lennie", "Bartholomew", "Katie",
+            "Maybelline", "Kimball", "Elvis", "Les", "Flick", "Harley", "Beth", "Bidelia", "Montague", "Helen", "Ozzy",
+            "Stef", "Debra", "Maxene", "Stefanie", "Russ", "Avril", "Johnathan", "Orson", "Chelsey", "Josephine",
+            "Deshaun", "Wendell", "Lula", "Ferdinanda", "Greg", "Brad", "Kynaston", "Dena", "Russel", "Robertina",
+            "Misti", "Leon", "Anjelica", "Bryana", "Myles", "Judi", "Curtis", "Davin", "Kristia", "Chrysanta",
+            "Hayleigh", "Hector", "Osbert", "Eustace", "Cary", "Tansy", "Cayley", "Maryann", "Alissa", "Ike",
+            "Tranter", "Reina", "Alwilda", "Sidony", "Columbine", "Astra", "Jillie", "Stephania", "Jonah", "Kennedy",
+            "Ferdinand", "Allegria", "Donella", "Kelleigh", "Darian", "Eldreda", "Jayden", "Herbie", "Jake", "Winston",
+            "Vi", "Annie", "Cherice", "Hugo", "Tricia", "Haydee", "Cassarah", "Darden", "Mallory", "Alton", "Hadley",
+            "Romayne", "Lacey", "Ern", "Alayna", "Cecilia", "Seward", "Tilly", "Edgar", "Concordia", "Ibbie", "Dahlia",
+            "Oswin", "Stu", "Brett", "Maralyn", "Kristeen", "Dotty", "Robyn", "Nessa", "Tresha", "Guinevere",
+            "Emerson", "Haze", "Lyn", "Henderson", "Lexa", "Jaylen", "Gail", "Lizette", "Tiara", "Robbie", "Destiny",
+            "Alice", "Livia", "Rosy", "Leah", "Jan", "Zach", "Vita", "Gia", "Micheal", "Rowina", "Alysha", "Bobbi",
+            "Delores", "Osmond", "Karaugh", "Wilbur", "Kasandra", "Renae", "Kaety", "Dora", "Gaye", "Amaryllis",
+            "Katelyn", "Dacre", "Prudence", "Ebony", "Camron", "Jerrold", "Vivyan", "Randall", "Donna", "Misty",
+            "Damon", "Selby", "Esmund", "Rian", "Garry", "Julius", "Raelene", "Clement", "Dom", "Tibby", "Moss",
+            "Millicent", "Gwendoline", "Berry", "Ashleigh", "Lilac", "Quin", "Vere", "Creighton", "Harriet", "Malvina",
+            "Lianne", "Pearle", "Kizzie", "Kara", "Petula", "Jeanie", "Maria", "Pacey", "Victoria", "Huey", "Toni",
+            "Rose", "Wallis", "Diggory", "Josiah", "Delma", "Keysha", "Channing", "Prue", "Lee", "Ryan", "Sidney",
+            "Valerie", "Clancy", "Ezra", "Gilbert", "Clare", "Laz", "Crofton", "Mike", "Annabella", "Tara", "Eldred",
+            "Arthur", "Jaylon", "Peronel", "Paden", "Dot", "Marian", "Amyas", "Alexus", "Esmond", "Abbie", "Stanley",
+            "Brittani", "Vickie", "Errol", "Kimberlee", "Uland", "Ebenezer", "Howie", "Eveline", "Andrea", "Trish",
+            "Hopkin", "Bryanna", "Temperance", "Valarie", "Femie", "Alix", "Terrell", "Lewin", "Lorrin", "Happy",
+            "Micah", "Rachyl", "Sloan", "Gertrude", "Elizabeth", "Dorris", "Andra", "Bram", "Gary", "Jeannine",
+            "Maurene", "Irene", "Yolonda", "Jonty", "Coleen", "Cecelia", "Chantal", "Stuart", "Caris", "Ros",
+            "Kaleigh", "Mirabelle", "Kolby", "Primrose", "Susannah", "Ginny", "Jinny", "Dolly", "Lettice", "Sonny",
+            "Melva", "Ernest", "Garret", "Reagan", "Trenton", "Gallagher", "Edwin", "Nikolas", "Corrie", "Lynette",
+            "Ettie", "Sly", "Debbi", "Eudora", "Brittney", "Tacey", "Marius", "Anima", "Gordon", "Olivia", "Kortney",
+            "Shantel", "Kolleen", "Nevaeh", "Buck", "Sera", "Liliana", "Aric", "Kalyn", "Mick", "Libby", "Ingram",
+            "Alexandria", "Darleen", "Jacklyn", "Hughie", "Tyler", "Aida", "Ronda", "Deemer", "Taryn", "Laureen",
+            "Samantha", "Dave", "Hardy", "Baldric", "Montgomery", "Gus", "Ellis", "Titania", "Luke", "Chase", "Haidee",
+            "Mayra", "Isabell", "Trinity", "Milo", "Abigail", "Tacita", "Meg", "Hervey", "Natasha", "Sadie", "Holden",
+            "Dee", "Mansel", "Perry", "Randi", "Frederica", "Georgina", "Kolour", "Debbie", "Seraphina", "Elspet",
+            "Julyan", "Raven", "Zavia", "Jarvis", "Jaymes", "Grover", "Cairo", "Alea", "Jordon", "Braxton", "Donny",
+            "Rhoda", "Tonya", "Bee", "Alyssia", "Ashlyn", "Reanna", "Lonny", "Arlene", "Deb", "Jane", "Nikole",
+            "Bettina", "Harrison", "Tamzen", "Arielle", "Adelaide", "Faith", "Bridie", "Wilburn", "Fern", "Nan",
+            "Shaw", "Zeke", "Alan", "Dene", "Gina", "Alexa", "Bailey", "Sal", "Tammy", "Maximillian", "America",
+            "Sylvana", "Fitz", "Mo", "Marissa", "Cass", "Eldon", "Wilfrid", "Tel", "Joann", "Kendra", "Tolly",
+            "Leanne", "Ferdie", "Haven", "Lucas", "Marlee", "Cyrilla", "Red", "Phoenix", "Jazmin", "Carin", "Gena",
+            "Lashonda", "Tucker", "Genette", "Kizzy", "Winifred", "Melody", "Keely", "Kaylyn", "Radcliff", "Lettie",
+            "Foster", "Lyndsey", "Nicholas", "Farley", "Louisa", "Dana", "Dortha", "Francine", "Doran", "Bonita",
+            "Hal", "Sawyer", "Reginald", "Aislin", "Nathan", "Baylee", "Abilene", "Ladonna", "Maurine", "Shelly",
+            "Deandre", "Jasmin", "Roderic", "Tiffany", "Amanda", "Verity", "Wilford", "Gayelord", "Whitney", "Demelza",
+            "Kenton", "Alberta", "Kyra", "Tabitha", "Sampson", "Korey", "Lillian", "Edison", "Clayton", "Steph",
+            "Maya", "Dusty", "Jim", "Ronny", "Adrianne", "Bernard", "Harris", "Kiley", "Alexander", "Kisha", "Ethalyn",
+            "Patience", "Briony", "Indigo", "Aureole", "Makenzie", "Molly", "Sherilyn", "Barry", "Laverne", "Hunter",
+            "Rocky", "Tyreek", "Madalyn", "Phyliss", "Chet", "Beatrice", "Faye", "Lavina", "Madelyn", "Tracey",
+            "Gyles", "Patti", "Carlyn", "Stephanie", "Jackalyn", "Larrie", "Kimmy", "Isolda", "Emelina", "Lis",
+            "Zillah", "Cody", "Sheard", "Rufus", "Paget", "Mae", "Rexanne", "Luvinia", "Tamsen", "Rosanna", "Greig",
+            "Stacia", "Mabelle", "Quianna", "Lotus", "Delice", "Bradford", "Angus", "Cosmo", "Earlene", "Adrian",
+            "Arlie", "Noelle", "Sabella", "Isa", "Adelle", "Innocent", "Kirby", "Trixie", "Kenelm", "Nelda", "Melia",
+            "Kendal", "Dorinda", "Placid", "Linette", "Kam", "Sherisse", "Evan", "Ewart", "Janice", "Linton",
+            "Jacaline", "Charissa", "Douglas", "Aileen", "Kemp", "Oli", "Amethyst", "Rosie", "Nigella", "Sherill",
+            "Anderson", "Alanna", "Eric", "Claudia", "Jennifer", "Boniface", "Harriet", "Vernon", "Lucy", "Shawnee",
+            "Gerard", "Cecily", "Romey", "Randall", "Wade", "Lux", "Dawson", "Gregg", "Kade", "Roxanne", "Melinda",
+            "Rolland", "Rowanne", "Fannie", "Isidore", "Melia", "Harvie", "Salal", "Eleonor", "Jacquette", "Lavone",
+            "Shanika", "Tarquin", "Janet", "Josslyn", "Maegan", "Augusta", "Aubree", "Francene", "Martie", "Marisa",
+            "Tyreek", "Tatianna", "Caleb", "Sheridan", "Nellie", "Barbara", "Wat", "Jayla", "Esmaralda", "Graeme",
+            "Lavena", "Jemima", "Nikolas", "Triston", "Portia", "Kyla", "Marcus", "Raeburn", "Jamison", "Earl", "Wren",
+            "Leighton", "Lagina", "Lucasta", "Dina", "Amaranta", "Jessika", "Claud", "Bernard", "Winifred", "Ebba",
+            "Sammi", "Gall", "Chloe", "Ottoline", "Herbert", "Janice", "Gareth", "Channing", "Caleigh", "Kailee",
+            "Ralphie", "Tamzen", "Quincy", "Beaumont", "Albert", "Jadyn", "Violet", "Luanna", "Moriah", "Humbert",
+            "Jed", "Leona", "Hale", "Mitch", "Marlin", "Nivek", "Darwin", "Dirk", "Liliana", "Meadow", "Bernadine",
+            "Jorie", "Peyton", "Astra", "Roscoe", "Gina", "Lovell", "Jewel", "Romayne", "Rosy", "Imogene",
+            "Margaretta", "Lorinda", "Hopkin", "Bobby", "Flossie", "Bennie", "Horatio", "Jonah", "Lyn", "Deana",
+            "Juliana", "Blanch", "Wright", "Kendal", "Woodrow", "Tania", "Austyn", "Val", "Mona", "Charla", "Rudyard",
+            "Pamela", "Raven", "Zena", "Nicola", "Kaelea", "Conor", "Virgil", "Sonnie", "Goodwin", "Christianne",
+            "Linford", "Myron", "Denton", "Charita", "Brody", "Ginnie", "Harrison", "Jeanine", "Quin", "Isolda",
+            "Zoie", "Pearce", "Margie", "Larrie", "Angelina", "Marcia", "Jessamine", "Delilah", "Dick", "Luana",
+            "Delicia", "Lake", "Luvenia", "Vaughan", "Concordia", "Gayelord", "Cheyenne", "Felix", "Dorris", "Pen",
+            "Kristeen", "Parris", "Everitt", "Josephina", "Amy", "Tommie", "Adrian", "April", "Rosaline", "Zachery",
+            "Trace", "Phoebe", "Jenelle", "Kameron", "Katharine", "Media", "Colton", "Tad", "Quianna", "Kerenza",
+            "Greta", "Luvinia", "Pete", "Tonya", "Beckah", "Barbra", "Jon", "Tetty", "Corey", "Sylvana", "Kizzy",
+            "Korey", "Trey", "Haydee", "Penny", "Mandy", "Panda", "Coline", "Ramsey", "Sukie", "Annabel", "Sarina",
+            "Corbin", "Suzanna", "Rob", "Duana", "Shell", "Jason", "Eddy", "Rube", "Roseann", "Celia", "Brianne",
+            "Nerissa", "Jera", "Humphry", "Ashlynn", "Terrence", "Philippina", "Coreen", "Kolour", "Indiana", "Paget",
+            "Marlyn", "Hester", "Isbel", "Ocean", "Harris", "Leslie", "Vere", "Monroe", "Isabelle", "Bertie", "Clitus",
+            "Dave", "Alethea", "Lessie", "Louiza", "Madlyn", "Garland", "Wolf", "Lalo", "Donny", "Amabel", "Tianna",
+            "Louie", "Susie", "Mackenzie", "Renie", "Tess", "Marmaduke", "Gwendolen", "Bettina", "Beatrix", "Esmund",
+            "Minnie", "Carlie", "Barnabas", "Ruthie", "Honour", "Haylie", "Xavior", "Freddie", "Ericka", "Aretha",
+            "Edie", "Madelina", "Anson", "Tabby", "Derrick", "Jocosa", "Deirdre", "Aislin", "Chastity", "Abigail",
+            "Wynonna", "Zo", "Eldon", "Krystine", "Ghislaine", "Zavia", "Nolene", "Marigold", "Kelley", "Sylvester",
+            "Odell", "George", "Laurene", "Franklyn", "Clarice", "Mo", "Dustin", "Debbi", "Lina", "Tony", "Acacia",
+            "Hettie", "Natalee", "Marcie", "Brittany", "Elnora", "Rachel", "Dawn", "Basil", "Christal", "Anjelica",
+            "Fran", "Tawny", "Delroy", "Tameka", "Lillie", "Ceara", "Deanna", "Deshaun", "Ken", "Bradford", "Justina",
+            "Merle", "Draven", "Gretta", "Harriette", "Webster", "Nathaniel", "Anemone", "Coleen", "Ruth", "Chryssa",
+            "Hortensia", "Saffie", "Deonne", "Leopold", "Harlan", "Lea", "Eppie", "Lucinda", "Tilda", "Fanny", "Titty",
+            "Lockie", "Jepson", "Sherisse", "Maralyn", "Ethel", "Sly", "Ebenezer", "Canute", "Ella", "Freeman",
+            "Reuben", "Olivette", "Nona", "Rik", "Amice", "Kristine", "Kathie", "Jayne", "Jeri", "Mckenna", "Bertram",
+            "Kaylee", "Livia", "Gil", "Wallace", "Maryann", "Keeleigh", "Laurinda", "Doran", "Khloe", "Dakota",
+            "Yaron", "Kimberleigh", "Gytha", "Doris", "Marylyn", "Benton", "Linnette", "Esther", "Jakki", "Rowina",
+            "Marian", "Roselyn", "Norbert", "Maggie", "Caesar", "Phinehas", "Jerry", "Jasmine", "Antonette", "Miriam",
+            "Monna", "Maryvonne", "Jacquetta", "Bernetta", "Napier", "Annie", "Gladwin", "Sheldon", "Aric", "Elouise",
+            "Gawain", "Kristia", "Gabe", "Kyra", "Red", "Tod", "Dudley", "Lorraine", "Ryley", "Sabina", "Poppy",
+            "Leland", "Aileen", "Eglantine", "Alicia", "Jeni", "Addy", "Tiffany", "Geffrey", "Lavina", "Collin",
+            "Clover", "Vin", "Jerome", "Doug", "Vincent", "Florence", "Scarlet", "Celeste", "Desdemona", "Tiphanie",
+            "Kassandra", "Ashton", "Madison", "Art", "Magdalene", "Iona", "Josepha", "Anise", "Ferne", "Derek",
+            "Huffie", "Qiana", "Ysabel", "Tami", "Shannah", "Xavier", "Willard", "Winthrop", "Vickie", "Maura",
+            "Placid", "Tiara", "Reggie", "Elissa", "Isa", "Chrysanta", "Jeff", "Bessie", "Terri", "Amilia", "Brett",
+            "Daniella", "Damion", "Carolina", "Maximillian", "Travers", "Benjamin", "Oprah", "Darcy", "Yolanda",
+            "Nicolina", "Crofton", "Jarrett", "Kaitlin", "Shauna", "Keren", "Bevis", "Kalysta", "Sharron", "Alyssa",
+            "Blythe", "Zelma", "Caelie", "Norwood", "Billie", "Patrick", "Gary", "Cambria", "Tylar", "Mason", "Helen",
+            "Melyssa", "Gene", "Gilberta", "Carter", "Herbie", "Harmonie", "Leola", "Eugenia", "Clint", "Pauletta",
+            "Edwyna", "Georgina", "Teal", "Harper", "Izzy", "Dillon", "Kezia", "Evangeline", "Colene", "Madelaine",
+            "Zilla", "Rudy", "Dottie", "Caris", "Morton", "Marge", "Tacey", "Parker", "Troy", "Liza", "Lewin",
+            "Tracie", "Justine", "Dallas", "Linden", "Ray", "Loretta", "Teri", "Elvis", "Diane", "Julianna", "Manfred",
+            "Denise", "Eireen", "Ann", "Kenith", "Linwood", "Kathlyn", "Bernice", "Shelley", "Oswald", "Amedeus",
+            "Homer", "Tanzi", "Ted", "Ralphina", "Hyacinth", "Lotus", "Matthias", "Arlette", "Clark", "Cecil",
+            "Elspeth", "Alvena", "Noah", "Millard", "Brenden", "Cole", "Philipa", "Nina", "Thelma", "Iantha", "Reid",
+            "Jefferson", "Meg", "Elsie", "Shirlee", "Nathan", "Nancy", "Simona", "Racheal", "Carin", "Emory", "Delice",
+            "Kristi", "Karaugh", "Kaety", "Tilly", "Em", "Alanis", "Darrin", "Jerrie", "Hollis", "Cary", "Marly",
+            "Carita", "Jody", "Farley", "Hervey", "Rosalin", "Cuthbert", "Stewart", "Jodene", "Caileigh", "Briscoe",
+            "Dolores", "Sheree", "Eustace", "Nigel", "Detta", "Barret", "Rowland", "Kenny", "Githa", "Zoey", "Adela",
+            "Petronella", "Opal", "Coleman", "Niles", "Cyril", "Dona", "Alberic", "Allannah", "Jules", "Avalon",
+            "Hadley", "Thomas", "Renita", "Calanthe", "Heron", "Shawnda", "Chet", "Malina", "Manny", "Rina", "Frieda",
+            "Eveleen", "Deshawn", "Amos", "Raelene", "Paige", "Molly", "Nannie", "Ileen", "Brendon", "Milford",
+            "Unice", "Rebeccah", "Caedmon", "Gae", "Doreen", "Vivian", "Louis", "Raphael", "Vergil", "Lise", "Glenn",
+            "Karyn", "Terance", "Reina", "Jake", "Gordon", "Wisdom", "Isiah", "Gervase", "Fern", "Marylou", "Roddy",
+            "Justy", "Derick", "Shantelle", "Adam", "Chantel", "Madoline", "Emmerson", "Lexie", "Mickey", "Stephen",
+            "Dane", "Stacee", "Elwin", "Tracey", "Alexandra", "Ricky", "Ian", "Kasey", "Rita", "Alanna", "Georgene",
+            "Deon", "Zavier", "Ophelia", "Deforest", "Lowell", "Zubin", "Hardy", "Osmund", "Tabatha", "Debby",
+            "Katlyn", "Tallulah", "Priscilla", "Braden", "Wil", "Keziah", "Jen", "Aggie", "Korbin", "Lemoine",
+            "Barnaby", "Tranter", "Goldie", "Roderick", "Trina", "Emery", "Pris", "Sidony", "Adelle", "Tate", "Wilf",
+            "Zola", "Brande", "Chris", "Calanthia", "Lilly", "Kaycee", "Lashonda", "Jasmin", "Elijah", "Shantel",
+            "Simon", "Rosalind", "Jarod", "Kaylie", "Corrine", "Joselyn", "Archibald", "Mariabella", "Winton",
+            "Merlin", "Chad", "Ursula", "Kristopher", "Hewie", "Adrianna", "Lyndsay", "Jasmyn", "Tim", "Evette",
+            "Margaret", "Samson", "Bronte", "Terence", "Leila", "Candice", "Tori", "Jamey", "Coriander", "Conrad",
+            "Floyd", "Karen", "Lorin", "Maximilian", "Cairo", "Emily", "Yasmin", "Karolyn", "Bryan", "Lanny",
+            "Kimberly", "Rick", "Chaz", "Krystle", "Lyric", "Laura", "Garrick", "Flip", "Monty", "Brendan",
+            "Ermintrude", "Rayner", "Merla", "Titus", "Marva", "Patricia", "Leone", "Tracy", "Jaqueline", "Hallam",
+            "Delores", "Cressida", "Carlyle", "Leann", "Kelcey", "Laurence", "Ryan", "Reynold", "Mark", "Collyn",
+            "Audie", "Sammy", "Ellery", "Sallie", "Pamelia", "Adolph", "Lydia", "Titania", "Ron", "Bridger", "Aline",
+            "Read", "Kelleigh", "Weldon", "Irving", "Garey", "Diggory", "Evander", "Kylee", "Deidre", "Ormond",
+            "Laurine", "Reannon", "Arline", "Pat"
+
+    };
+
+    public static String[] jargon = { "wireless", "signal", "network", "3G", "plan", "touch-screen",
+            "customer-service", "reachability", "voice-command", "shortcut-menu", "customization", "platform", "speed",
+            "voice-clarity", "voicemail-service" };
+
+    public static String[] vendors = { "at&t", "verizon", "t-mobile", "sprint", "motorola", "samsung", "iphone" };
+
+    public static String[] org_list = { "Latsonity", "ganjalax", "Zuncan", "Lexitechno", "Hot-tech", "subtam",
+            "Coneflex", "Ganjatax", "physcane", "Tranzap", "Qvohouse", "Zununoing", "jaydax", "Keytech", "goldendexon",
+            "Villa-tech", "Trustbam", "Newcom", "Voltlane", "Ontohothex", "Ranhotfan", "Alphadax", "Transhigh",
+            "kin-ron", "Doublezone", "Solophase", "Vivaace", "silfind", "Basecone", "sonstreet", "Freshfix",
+            "Techitechi", "Kanelectrics", "linedexon", "Goldcity", "Newfase", "Technohow", "Zimcone", "Salthex",
+            "U-ron", "Solfix", "whitestreet", "Xx-technology", "Hexviafind", "over-it", "Strongtone", "Tripplelane",
+            "geomedia", "Scotcity", "Inchex", "Vaiatech", "Striptaxon", "Hatcom", "tresline", "Sanjodax", "freshdox",
+            "Sumlane", "Quadlane", "Newphase", "overtech", "Voltbam", "Icerunin", "Fixdintex", "Hexsanhex", "Statcode",
+            "Greencare", "U-electrics", "Zamcorporation", "Ontotanin", "Tanzimcare", "Groovetex", "Ganjastrip",
+            "Redelectronics", "Dandamace", "Whitemedia", "strongex", "Streettax", "highfax", "Mathtech", "Xx-drill",
+            "Sublamdox", "Unijobam", "Rungozoom", "Fixelectrics", "Villa-dox", "Ransaofan", "Plexlane", "itlab",
+            "Lexicone", "Fax-fax", "Viatechi", "Inchdox", "Kongreen", "Doncare", "Y-geohex", "Opeelectronics",
+            "Medflex", "Dancode", "Roundhex", "Labzatron", "Newhotplus", "Sancone", "Ronholdings", "Quoline",
+            "zoomplus", "Fix-touch", "Codetechno", "Tanzumbam", "Indiex", "Canline" };
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java
new file mode 100644
index 0000000..b2ba88e
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.Iterator;
+
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+
+public class DataGeneratorDriver {
+
+    public static void main(String[] args) {
+
+        DataGenerator.initialize(new InitializationInfo());
+        Iterator<TweetMessage> tweetIterator = DataGenerator.getTwitterMessageIterator();
+        while (tweetIterator.hasNext()) {
+            System.out.println(tweetIterator.next().toString());
+        }
+    }
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java
new file mode 100644
index 0000000..a536076
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.dataset.adapter.StreamBasedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class GenericSocketFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final String KEY_PORT = "port";
+
+    private static final Logger LOGGER = Logger.getLogger(GenericSocketFeedAdapter.class.getName());
+
+    private Map<String, String> configuration;
+
+    private SocketFeedServer socketFeedServer;
+
+    private static final int DEFAULT_PORT = 2909;
+
+    public GenericSocketFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
+            ARecordType outputtype, IHyracksTaskContext ctx) throws AsterixException, IOException {
+        super(parserFactory, outputtype, ctx);
+        this.configuration = configuration;
+        String portValue = (String) this.configuration.get(KEY_PORT);
+        int port = portValue != null ? Integer.parseInt(portValue) : DEFAULT_PORT;
+        this.socketFeedServer = new SocketFeedServer(configuration, outputtype, port);
+    }
+
+    @Override
+    public void start(int partition, IFrameWriter writer) throws Exception {
+        super.start(partition, writer);
+    }
+
+    @Override
+    public InputStream getInputStream(int partition) throws IOException {
+        return socketFeedServer.getInputStream();
+    }
+
+    private static class SocketFeedServer {
+        private ServerSocket serverSocket;
+        private InputStream inputStream;
+
+        public SocketFeedServer(Map<String, String> configuration, ARecordType outputtype, int port)
+                throws IOException, AsterixException {
+            try {
+                serverSocket = new ServerSocket(port);
+            } catch (Exception e) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("port: " + port + " unusable ");
+                }
+            }
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Feed server configured to use port: " + port);
+            }
+        }
+
+        public InputStream getInputStream() {
+            Socket socket;
+            try {
+                socket = serverSocket.accept();
+                inputStream = socket.getInputStream();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+            return inputStream;
+        }
+
+        public void stop() throws IOException {
+            serverSocket.close();
+        }
+
+    }
+
+    @Override
+    public void stop() throws Exception {
+        socketFeedServer.stop();
+    }
+
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
new file mode 100644
index 0000000..788bb4f
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
@@ -0,0 +1,74 @@
+/*
+x * Copyright 2009-2012 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.tools.external.data;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{GenericSocketFeedAdapter} The
+ * adapter listens at a port for receiving data (from external world).
+ * Data received is transformed into Asterix Data Format (ADM) and stored into
+ * a dataset a configured in the Adapter configuration.
+ */
+public class GenericSocketFeedAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    private ARecordType outputType;
+
+    @Override
+    public String getName() {
+        return "generic_socket_feed";
+    }
+
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.GENERIC;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+        this.configuration = configuration;
+        outputType = (ARecordType) outputType;
+        this.configureFormat(outputType);
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(1);
+    }
+
+    @Override
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        return new GenericSocketFeedAdapter(configuration, parserFactory, outputType, ctx);
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index e037ec6..a92cb55 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -18,43 +18,28 @@
 import java.io.InputStream;
 import java.util.Map;
 
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
-import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
-import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
-import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
- * An adapter that simulates a feed from the contents of a source file. The file can be on the local file
- * system or on HDFS. The feed ends when the content of the source file has been ingested.
+ * An adapter that simulates a feed from the contents of a source file. The file
+ * can be on the local file system or on HDFS. The feed ends when the content of
+ * the source file has been ingested.
  */
-public class RateControlledFileSystemBasedAdapter extends FileSystemBasedAdapter implements ITypedDatasourceAdapter,
-        IManagedFeedAdapter {
+
+public class RateControlledFileSystemBasedAdapter extends FileSystemBasedAdapter implements IFeedAdapter {
 
     private static final long serialVersionUID = 1L;
     private FileSystemBasedAdapter coreAdapter;
-    private String format;
 
-    public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, Object> configuration,
-            FileSystemBasedAdapter coreAdapter, String format) throws Exception {
-        super(atype);
-        this.configuration = configuration;
+    public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, String> configuration,
+            FileSystemBasedAdapter coreAdapter, String format, ITupleParserFactory parserFactory,
+            IHyracksTaskContext ctx) throws Exception {
+        super(parserFactory, atype, ctx);
         this.coreAdapter = coreAdapter;
-        this.format = format;
     }
 
     @Override
@@ -63,184 +48,8 @@
     }
 
     @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        coreAdapter.initialize(ctx);
-        this.ctx = ctx;
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        coreAdapter.configure(arguments);
-    }
-
-    @Override
-    public AdapterType getAdapterType() {
-        return coreAdapter.getAdapterType();
-    }
-
-    @Override
-    protected ITupleParser getTupleParser() throws Exception {
-        ITupleParser parser = null;
-        if (format.equals(FORMAT_DELIMITED_TEXT)) {
-            parser = getRateControlledDelimitedDataTupleParser((ARecordType) atype);
-        } else if (format.equals(FORMAT_ADM)) {
-            parser = getRateControlledADMDataTupleParser((ARecordType) atype);
-        } else {
-            throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
-        }
-        return parser;
-
-    }
-
-    protected ITupleParser getRateControlledDelimitedDataTupleParser(ARecordType recordType) throws AsterixException,
-            HyracksDataException {
-        ITupleParser parser;
-        int n = recordType.getFieldTypes().length;
-        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
-        for (int i = 0; i < n; i++) {
-            ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
-            IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
-            if (vpf == null) {
-                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-            }
-            fieldParserFactories[i] = vpf;
-
-        }
-        String delimiterValue = (String) configuration.get(KEY_DELIMITER);
-        if (delimiterValue != null && delimiterValue.length() > 1) {
-            throw new AsterixException("improper delimiter");
-        }
-
-        Character delimiter = delimiterValue.charAt(0);
-        parser = new RateControlledTupleParserFactory(recordType, fieldParserFactories, delimiter, configuration)
-                .createTupleParser(ctx);
-        return parser;
-    }
-
-    protected ITupleParser getRateControlledADMDataTupleParser(ARecordType recordType) throws AsterixException {
-        ITupleParser parser = null;
-        try {
-            parser = new RateControlledTupleParserFactory(recordType, configuration).createTupleParser(ctx);
-            return parser;
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return (ARecordType) atype;
-    }
-
-    @Override
-    public void alter(Map<String, String> properties) {
-        ((RateControlledTupleParser) parser).setInterTupleInterval(Long.parseLong(properties
-                .get(RateControlledTupleParser.INTER_TUPLE_INTERVAL)));
-    }
-
-    @Override
     public void stop() {
-        ((RateControlledTupleParser) parser).stop();
+        ((RateControlledTupleParser) tupleParser).stop();
     }
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return coreAdapter.getPartitionConstraint();
-    }
-}
-
-class RateControlledTupleParserFactory implements ITupleParserFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final ARecordType recordType;
-    private final IDataParser dataParser;
-    private final Map<String, Object> configuration;
-
-    public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
-            char fieldDelimiter, Map<String, Object> configuration) {
-        this.recordType = recordType;
-        dataParser = new DelimitedDataParser(recordType, valueParserFactories, fieldDelimiter);
-        this.configuration = configuration;
-    }
-
-    public RateControlledTupleParserFactory(ARecordType recordType, Map<String, Object> configuration) {
-        this.recordType = recordType;
-        dataParser = new ADMDataParser();
-        this.configuration = configuration;
-    }
-
-    @Override
-    public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
-        return new RateControlledTupleParser(ctx, recordType, dataParser, configuration);
-    }
-
-}
-
-class RateControlledTupleParser extends AbstractTupleParser {
-
-    private final IDataParser dataParser;
-    private long interTupleInterval;
-    private boolean delayConfigured;
-    private boolean continueIngestion = true;
-
-    public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
-
-    public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
-            Map<String, Object> configuration) throws HyracksDataException {
-        super(ctx, recType);
-        this.dataParser = dataParser;
-        String propValue = (String) configuration.get(INTER_TUPLE_INTERVAL);
-        if (propValue != null) {
-            interTupleInterval = Long.parseLong(propValue);
-        } else {
-            interTupleInterval = 0;
-        }
-        delayConfigured = interTupleInterval != 0;
-    }
-
-    public void setInterTupleInterval(long val) {
-        this.interTupleInterval = val;
-        this.delayConfigured = val > 0;
-    }
-
-    public void stop() {
-        continueIngestion = false;
-    }
-
-    @Override
-    public IDataParser getDataParser() {
-        return dataParser;
-    }
-
-    @Override
-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-
-        appender.reset(frame, true);
-        IDataParser parser = getDataParser();
-        try {
-            parser.initialize(in, recType, true);
-            while (continueIngestion) {
-                tb.reset();
-                if (!parser.parse(tb.getDataOutput())) {
-                    break;
-                }
-                tb.addFieldEndOffset();
-                if (delayConfigured) {
-                    Thread.sleep(interTupleInterval);
-                }
-                addTupleToFrame(writer);
-            }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
-        } catch (AsterixException ae) {
-            throw new HyracksDataException(ae);
-        } catch (IOException ioe) {
-            throw new HyracksDataException(ioe);
-        } catch (InterruptedException ie) {
-            throw new HyracksDataException(ie);
-        }
-    }
 }
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index bf3c086..40bc814 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -14,14 +14,32 @@
  */
 package edu.uci.ics.asterix.tools.external.data;
 
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.Map;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
 import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
  * Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
@@ -29,38 +47,25 @@
  * on the local file system or on HDFS. The feed ends when the content of the
  * source file has been ingested.
  */
-public class RateControlledFileSystemBasedAdapterFactory implements IGenericDatasetAdapterFactory {
+public class RateControlledFileSystemBasedAdapterFactory extends StreamBasedAdapterFactory implements
+        IGenericAdapterFactory {
     private static final long serialVersionUID = 1L;
-    
+
     public static final String KEY_FILE_SYSTEM = "fs";
     public static final String LOCAL_FS = "localfs";
     public static final String HDFS = "hdfs";
     public static final String KEY_PATH = "path";
     public static final String KEY_FORMAT = "format";
 
-    private IGenericDatasetAdapterFactory adapterFactory;
+    private IGenericAdapterFactory adapterFactory;
     private String format;
-    private boolean setup = false;
+    private Map<String, String> configuration;
+    private ARecordType atype;
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType type) throws Exception {
-        if (!setup) {
-            checkRequiredArgs(configuration);
-            String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
-            String adapterFactoryClass = null;
-            if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
-                adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory";
-            } else if (fileSystem.equals(HDFS)) {
-                adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory";
-            } else {
-                throw new AsterixException("Unsupported file system type " + fileSystem);
-            }
-            format = (String) configuration.get(KEY_FORMAT);
-            adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
-            setup = true;
-        }
-        return new RateControlledFileSystemBasedAdapter((ARecordType) type, configuration,
-                (FileSystemBasedAdapter) adapterFactory.createAdapter(configuration, type), format);
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        FileSystemBasedAdapter coreAdapter = (FileSystemBasedAdapter) adapterFactory.createAdapter(ctx);
+        return new RateControlledFileSystemBasedAdapter(atype, configuration, coreAdapter, format, parserFactory, ctx);
     }
 
     @Override
@@ -68,11 +73,11 @@
         return "file_feed";
     }
 
-    private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
+    private void checkRequiredArgs(Map<String, String> configuration) throws Exception {
         if (configuration.get(KEY_FILE_SYSTEM) == null) {
             throw new Exception("File system type not specified. (fs=?) File system could be 'localfs' or 'hdfs'");
         }
-        if (configuration.get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME) == null) {
+        if (configuration.get(KEY_SOURCE_DATATYPE) == null) {
             throw new Exception("Record type not specified (output-type-name=?)");
         }
         if (configuration.get(KEY_PATH) == null) {
@@ -83,4 +88,186 @@
         }
     }
 
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.GENERIC;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType) throws Exception {
+        this.configuration = configuration;
+        checkRequiredArgs(configuration);
+        String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
+        String adapterFactoryClass = null;
+        if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
+            adapterFactoryClass = NCFileSystemAdapterFactory.class.getName();
+        } else if (fileSystem.equals(HDFS)) {
+            adapterFactoryClass = HDFSAdapterFactory.class.getName();
+        } else {
+            throw new AsterixException("Unsupported file system type " + fileSystem);
+        }
+        format = configuration.get(KEY_FORMAT);
+        adapterFactory = (IGenericAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
+        adapterFactory.configure(configuration, recordType);
+
+        atype = (ARecordType) recordType;
+        configureFormat();
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return adapterFactory.getPartitionConstraint();
+    }
+
+    private void configureFormat() throws AsterixException {
+        switch (format) {
+            case FORMAT_ADM:
+                parserFactory = new RateControlledTupleParserFactory(atype, configuration);
+                break;
+
+            case FORMAT_DELIMITED_TEXT:
+                String delimiterValue = (String) configuration.get(KEY_DELIMITER);
+                if (delimiterValue != null && delimiterValue.length() > 1) {
+                    throw new AsterixException("improper delimiter");
+                }
+                IValueParserFactory[] valueParserFactories = getValueParserFactories(atype);
+                parserFactory = new RateControlledTupleParserFactory(atype, valueParserFactories,
+                        delimiterValue.charAt(0), configuration);
+                break;
+        }
+    }
+
+    protected IValueParserFactory[] getValueParserFactories(ARecordType recordType) throws AsterixException {
+        int n = recordType.getFieldTypes().length;
+        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+        for (int i = 0; i < n; i++) {
+            ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+            IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+            if (vpf == null) {
+                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+            }
+            fieldParserFactories[i] = vpf;
+
+        }
+        return fieldParserFactories;
+    }
+
+}
+
+class RateControlledTupleParserFactory implements ITupleParserFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ARecordType recordType;
+    private final Map<String, String> configuration;
+    private IValueParserFactory[] valueParserFactories;
+    private char delimiter;
+    private final ParserType parserType;
+
+    public enum ParserType {
+        ADM,
+        DELIMITED_DATA
+    }
+
+    public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
+            char fieldDelimiter, Map<String, String> configuration) {
+        this.recordType = recordType;
+        this.valueParserFactories = valueParserFactories;
+        this.delimiter = fieldDelimiter;
+        this.configuration = configuration;
+        this.parserType = ParserType.DELIMITED_DATA;
+    }
+
+    public RateControlledTupleParserFactory(ARecordType recordType, Map<String, String> configuration) {
+        this.recordType = recordType;
+        this.configuration = configuration;
+        this.parserType = ParserType.ADM;
+    }
+
+    @Override
+    public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+        IDataParser dataParser = null;
+        switch (parserType) {
+            case ADM:
+                dataParser = new ADMDataParser();
+                break;
+            case DELIMITED_DATA:
+                dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+                break;
+        }
+        return new RateControlledTupleParser(ctx, recordType, dataParser, configuration);
+    }
+
+}
+
+class RateControlledTupleParser extends AbstractTupleParser {
+
+    private final IDataParser dataParser;
+    private long interTupleInterval;
+    private boolean delayConfigured;
+    private boolean continueIngestion = true;
+
+    public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
+
+    public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
+            Map<String, String> configuration) throws HyracksDataException {
+        super(ctx, recType);
+        this.dataParser = dataParser;
+        String propValue = configuration.get(INTER_TUPLE_INTERVAL);
+        if (propValue != null) {
+            interTupleInterval = Long.parseLong(propValue);
+        } else {
+            interTupleInterval = 0;
+        }
+        delayConfigured = interTupleInterval != 0;
+    }
+
+    public void setInterTupleInterval(long val) {
+        this.interTupleInterval = val;
+        this.delayConfigured = val > 0;
+    }
+
+    public void stop() {
+        continueIngestion = false;
+    }
+
+    @Override
+    public IDataParser getDataParser() {
+        return dataParser;
+    }
+
+    @Override
+    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+
+        appender.reset(frame, true);
+        IDataParser parser = getDataParser();
+        try {
+            parser.initialize(in, recType, true);
+            while (continueIngestion) {
+                tb.reset();
+                if (!parser.parse(tb.getDataOutput())) {
+                    break;
+                }
+                tb.addFieldEndOffset();
+                if (delayConfigured) {
+                    Thread.sleep(interTupleInterval);
+                }
+                addTupleToFrame(writer);
+            }
+            if (appender.getTupleCount() > 0) {
+                FrameUtils.flushFrame(frame, writer);
+            }
+        } catch (AsterixException ae) {
+            throw new HyracksDataException(ae);
+        } catch (IOException ioe) {
+            throw new HyracksDataException(ioe);
+        } catch (InterruptedException ie) {
+            throw new HyracksDataException(ie);
+        }
+    }
 }
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java
new file mode 100644
index 0000000..d26e764
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java
@@ -0,0 +1,214 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedFeedClient;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.Message;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessageIterator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * TPS can be configured between 1 and 20,000
+ * 
+ * @author ramang
+ */
+public class SyntheticTwitterFeedAdapter extends PullBasedAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(SyntheticTwitterFeedAdapter.class.getName());
+
+    private Map<String, String> configuration;
+
+    public SyntheticTwitterFeedAdapter(Map<String, String> configuration, ARecordType outputType,
+            IHyracksTaskContext ctx) throws AsterixException {
+        super(configuration, ctx);
+        this.configuration = configuration;
+        this.adapterOutputType = outputType;
+    }
+
+    @Override
+    public IPullBasedFeedClient getFeedClient(int partition) throws Exception {
+        return new SyntheticTwitterFeedClient(configuration, adapterOutputType, partition);
+    }
+
+    private static class SyntheticTwitterFeedClient extends PullBasedFeedClient implements IPullBasedFeedClient {
+
+        private static final Logger LOGGER = Logger.getLogger(SyntheticTwitterFeedClient.class.getName());
+
+        public static final String KEY_DURATION = "duration";
+        public static final String KEY_TPS = "tps";
+        public static final String KEY_EXCEPTION_PERIOD = "exception-period";
+
+        private int duration;
+        private long tweetInterval;
+        private int numTweetsBeforeDelay;
+        private TweetMessageIterator tweetIterator = null;
+        private long exeptionInterval;
+
+        private IAObject[] mutableFields;
+        private ARecordType outputRecordType;
+        private int partition;
+        private int tweetCount = 0;
+        private int tweetCountBeforeException = 0;
+        private int exceptionPeriod = -1;
+
+        public SyntheticTwitterFeedClient(Map<String, String> configuration, ARecordType outputRecordType, int partition)
+                throws AsterixException {
+            this.outputRecordType = outputRecordType;
+            String value = (String) configuration.get(KEY_DURATION);
+            duration = value != null ? Integer.parseInt(value) : 60;
+            initializeTweetRate((String) configuration.get(KEY_TPS));
+            value = (String) configuration.get(KEY_EXCEPTION_PERIOD);
+            if (value != null) {
+                exceptionPeriod = Integer.parseInt(value);
+            }
+
+            InitializationInfo info = new InitializationInfo();
+            info.timeDurationInSecs = duration;
+            DataGenerator.initialize(info);
+            tweetIterator = new TweetMessageIterator(duration);
+            initialize();
+        }
+
+        private void initializeTweetRate(String tps) {
+            numTweetsBeforeDelay = 0;
+            if (tps == null) {
+                tweetInterval = 0;
+            } else {
+                int val = Integer.parseInt(tps);
+                double interval = new Double(((double) 1000 / val));
+                if (interval > 1) {
+                    tweetInterval = (long) interval;
+                    numTweetsBeforeDelay = 1;
+                } else {
+                    tweetInterval = 1;
+                    Double numTweets = new Double(1 / interval);
+                    if (numTweets.intValue() != numTweets) {
+                        tweetInterval = 10;
+                        numTweetsBeforeDelay = (new Double(10 * numTweets * 1.4)).intValue();
+                    } else {
+                        numTweetsBeforeDelay = new Double((numTweets * 1.4)).intValue();
+                    }
+                }
+            }
+
+        }
+
+        private void writeTweet(TweetMessage next) {
+
+            // tweet id
+            LOGGER.info("Generating next tweet");
+            ((AMutableString) mutableFields[0]).setValue(next.getTweetid());
+            mutableRecord.setValueAtPos(0, mutableFields[0]);
+
+            // user
+            AMutableRecord userRecord = ((AMutableRecord) mutableFields[1]);
+            ((AMutableString) userRecord.getValueByPos(0)).setValue(next.getUser().getScreenName());
+            ((AMutableString) userRecord.getValueByPos(1)).setValue("en");
+            ((AMutableInt32) userRecord.getValueByPos(2)).setValue(next.getUser().getFriendsCount());
+            ((AMutableInt32) userRecord.getValueByPos(3)).setValue(next.getUser().getStatusesCount());
+            ((AMutableString) userRecord.getValueByPos(4)).setValue(next.getUser().getName());
+            ((AMutableInt32) userRecord.getValueByPos(5)).setValue(next.getUser().getFollowersCount());
+            mutableRecord.setValueAtPos(1, userRecord);
+
+            // location
+            ((AMutablePoint) mutableFields[2]).setValue(next.getSenderLocation().getLatitude(), next
+                    .getSenderLocation().getLongitude());
+            mutableRecord.setValueAtPos(2, mutableFields[2]);
+
+            // time
+            ((AMutableDateTime) mutableFields[3]).setValue(next.getSendTime().getChrononTime());
+            mutableRecord.setValueAtPos(3, mutableFields[3]);
+
+            // referred topics
+            ((AMutableUnorderedList) mutableFields[4]).clear();
+            List<String> referredTopics = next.getReferredTopics();
+            for (String topic : referredTopics) {
+                ((AMutableUnorderedList) mutableFields[4]).add(new AMutableString(topic));
+            }
+            mutableRecord.setValueAtPos(4, mutableFields[4]);
+
+            // text
+            Message m = next.getMessageText();
+            char[] content = m.getMessage();
+            String tweetText = new String(content, 0, m.getLength());
+            ((AMutableString) mutableFields[5]).setValue(tweetText);
+            mutableRecord.setValueAtPos(5, mutableFields[5]);
+            LOGGER.info(tweetText);
+
+        }
+
+        @Override
+        public void resetOnFailure(Exception e) throws AsterixException {
+            // TODO Auto-generated method stub
+
+        }
+
+        @Override
+        public boolean alter(Map<String, String> configuration) {
+            // TODO Auto-generated method stub
+            return false;
+        }
+
+        @Override
+        public InflowState setNextRecord() throws Exception {
+            boolean moreData = tweetIterator.hasNext();
+            if (!moreData) {
+                return InflowState.NO_MORE_DATA;
+            }
+            writeTweet(tweetIterator.next());
+            if (tweetInterval != 0) {
+                tweetCount++;
+                if (tweetCount == numTweetsBeforeDelay) {
+                    Thread.sleep(tweetInterval);
+                    tweetCount = 0;
+                }
+            }
+            tweetCountBeforeException++;
+
+            if (tweetCountBeforeException == exceptionPeriod) {
+                tweetCountBeforeException = 0;
+                throw new AsterixException("Delibrate exception");
+            }
+            return InflowState.DATA_AVAILABLE;
+        }
+
+        private void initialize() throws AsterixException {
+            ARecordType userRecordType = (ARecordType) outputRecordType.getFieldTypes()[1];
+            IAObject[] userMutableFields = new IAObject[] { new AMutableString(""), new AMutableString(""),
+                    new AMutableInt32(0), new AMutableInt32(0), new AMutableString(""), new AMutableInt32(0) };
+            AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+            mutableFields = new IAObject[] { new AMutableString(""),
+                    new AMutableRecord(userRecordType, userMutableFields), new AMutablePoint(0, 0),
+                    new AMutableDateTime(0), new AMutableUnorderedList(unorderedListType), new AMutableString("") };
+            recordSerDe = new ARecordSerializerDeserializer(outputRecordType);
+            mutableRecord = new AMutableRecord(outputRecordType, mutableFields);
+
+        }
+
+        @Override
+        public void stop() {
+            // TODO Auto-generated method stub
+
+        }
+    }
+}
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
new file mode 100644
index 0000000..11cbe54
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
@@ -0,0 +1,141 @@
+/*
+x * Copyright 2009-2012 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.tools.external.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+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.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;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
+ * adapter simulates a feed from the contents of a source file. The file can be
+ * on the local file system or on HDFS. The feed ends when the content of the
+ * source file has been ingested.
+ */
+public class SyntheticTwitterFeedAdapterFactory implements ITypedAdapterFactory {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    private Map<String, String> configuration;
+
+    private static final String KEY_DATAVERSE_DATASET = "dataverse-dataset";
+
+    private static final ARecordType outputType = initOutputType();
+
+    @Override
+    public String getName() {
+        return "synthetic_twitter_feed";
+    }
+
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.TYPED;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        String dvds = (String) configuration.get(KEY_DATAVERSE_DATASET);
+        String[] components = dvds.split(":");
+        String dataverse = components[0];
+        String dataset = components[1];
+        MetadataTransactionContext ctx = null;
+        NodeGroup ng = null;
+        try {
+            ctx = MetadataManager.INSTANCE.beginTransaction();
+            Dataset ds = MetadataManager.INSTANCE.getDataset(ctx, dataverse, dataset);
+            String nodegroupName = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
+            ng = MetadataManager.INSTANCE.getNodegroup(ctx, nodegroupName);
+            MetadataManager.INSTANCE.commitTransaction(ctx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(ctx);
+            throw e;
+        }
+        List<String> storageNodes = ng.getNodeNames();
+        Set<String> nodes = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames();
+        String ingestionLocation = null;
+        if (nodes.size() > storageNodes.size()) {
+            nodes.removeAll(storageNodes);
+        }
+        String[] nodesArray = nodes.toArray(new String[] {});
+        Random r = new Random();
+        ingestionLocation = nodesArray[r.nextInt(nodes.size())];
+        return new AlgebricksAbsolutePartitionConstraint(new String[] { ingestionLocation });
+    }
+
+    @Override
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        return new SyntheticTwitterFeedAdapter(configuration, outputType, ctx);
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return outputType;
+    }
+
+    private static ARecordType initOutputType() {
+        ARecordType outputType = null;
+        try {
+            String[] userFieldNames = new String[] { "screen-name", "lang", "friends_count", "statuses_count", "name",
+                    "followers_count" };
+
+            IAType[] userFieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+                    BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT32 };
+            ARecordType userRecordType = new ARecordType("TwitterUserType", userFieldNames, userFieldTypes, false);
+
+            String[] fieldNames = new String[] { "tweetid", "user", "sender-location", "send-time", "referred-topics",
+                    "message-text" };
+
+            AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+            IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, userRecordType, BuiltinType.APOINT,
+                    BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
+            outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
+
+        } catch (AsterixException e) {
+            throw new IllegalStateException("Unable to initialize output type");
+        }
+        return outputType;
+    }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java
new file mode 100644
index 0000000..9d47de2
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java
@@ -0,0 +1,213 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedFeedClient;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.Message;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessageIterator;
+
+public class TweetGenerator extends PullBasedFeedClient implements IPullBasedFeedClient {
+
+    private static final Logger LOGGER = Logger.getLogger(TweetGenerator.class.getName());
+
+    public static final String KEY_DURATION = "duration";
+    public static final String KEY_TPS = "tps";
+    public static final String KEY_EXCEPTION_PERIOD = "exception-period";
+    public static final String OUTPUT_FORMAT = "output-format";
+
+    public static final String OUTPUT_FORMAT_ARECORD = "arecord";
+    public static final String OUTPUT_FORMAT_ADM_STRING = "adm-string";
+
+    private int duration;
+    private long tweetInterval;
+    private int numTweetsBeforeDelay;
+    private TweetMessageIterator tweetIterator = null;
+    private long exeptionInterval;
+
+    private IAObject[] mutableFields;
+    private ARecordType outputRecordType;
+    private int partition;
+    private int tweetCount = 0;
+    private int tweetCountBeforeException = 0;
+    private int exceptionPeriod = -1;
+    private boolean isOutputFormatRecord = false;
+    private byte[] EOL = "\n".getBytes();
+    private OutputStream os;
+
+    public TweetGenerator(Map<String, String> configuration, ARecordType outputRecordType, int partition, String format)
+            throws AsterixException {
+        this.outputRecordType = outputRecordType;
+        String value = configuration.get(KEY_DURATION);
+        duration = value != null ? Integer.parseInt(value) : 60;
+        initializeTweetRate(configuration.get(KEY_TPS));
+        value = configuration.get(KEY_EXCEPTION_PERIOD);
+        if (value != null) {
+            exceptionPeriod = Integer.parseInt(value);
+        }
+        isOutputFormatRecord = format.equalsIgnoreCase(OUTPUT_FORMAT_ARECORD);
+        InitializationInfo info = new InitializationInfo();
+        info.timeDurationInSecs = duration;
+        DataGenerator.initialize(info);
+        tweetIterator = new TweetMessageIterator(duration);
+        initialize();
+    }
+
+    private void initializeTweetRate(String tps) {
+        numTweetsBeforeDelay = 0;
+        if (tps == null) {
+            tweetInterval = 0;
+        } else {
+            int val = Integer.parseInt(tps);
+            double interval = new Double(((double) 1000 / val));
+            if (interval > 1) {
+                tweetInterval = (long) interval;
+                numTweetsBeforeDelay = 1;
+            } else {
+                tweetInterval = 1;
+                Double numTweets = new Double(1 / interval);
+                if (numTweets.intValue() != numTweets) {
+                    tweetInterval = 10;
+                    numTweetsBeforeDelay = (new Double(10 * numTweets * 1.4)).intValue();
+                } else {
+                    numTweetsBeforeDelay = new Double((numTweets * 1.4)).intValue();
+                }
+            }
+        }
+
+    }
+
+    private void writeTweetString(TweetMessage next) throws IOException {
+        String tweet = next.toString();
+        os.write(tweet.getBytes());
+        os.write(EOL);
+        /*
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(tweet);
+        }*/
+    }
+
+    private void writeTweetRecord(TweetMessage next) {
+
+        //tweet id
+        ((AMutableString) mutableFields[0]).setValue(next.getTweetid());
+        mutableRecord.setValueAtPos(0, mutableFields[0]);
+
+        // user 
+        AMutableRecord userRecord = ((AMutableRecord) mutableFields[1]);
+        ((AMutableString) userRecord.getValueByPos(0)).setValue(next.getUser().getScreenName());
+        ((AMutableString) userRecord.getValueByPos(1)).setValue("en");
+        ((AMutableInt32) userRecord.getValueByPos(2)).setValue(next.getUser().getFriendsCount());
+        ((AMutableInt32) userRecord.getValueByPos(3)).setValue(next.getUser().getStatusesCount());
+        ((AMutableString) userRecord.getValueByPos(4)).setValue(next.getUser().getName());
+        ((AMutableInt32) userRecord.getValueByPos(5)).setValue(next.getUser().getFollowersCount());
+        mutableRecord.setValueAtPos(1, userRecord);
+
+        // location
+        ((AMutablePoint) mutableFields[2]).setValue(next.getSenderLocation().getLatitude(), next.getSenderLocation()
+                .getLongitude());
+        mutableRecord.setValueAtPos(2, mutableFields[2]);
+
+        // time
+        ((AMutableDateTime) mutableFields[3]).setValue(next.getSendTime().getChrononTime());
+        mutableRecord.setValueAtPos(3, mutableFields[3]);
+
+        // referred topics
+        ((AMutableUnorderedList) mutableFields[4]).clear();
+        List<String> referredTopics = next.getReferredTopics();
+        for (String topic : referredTopics) {
+            ((AMutableUnorderedList) mutableFields[4]).add(new AMutableString(topic));
+        }
+        mutableRecord.setValueAtPos(4, mutableFields[4]);
+
+        // text
+        Message m = next.getMessageText();
+        char[] content = m.getMessage();
+        String tweetText = new String(content, 0, m.getLength());
+        ((AMutableString) mutableFields[5]).setValue(tweetText);
+        mutableRecord.setValueAtPos(5, mutableFields[5]);
+        LOGGER.info(tweetText);
+
+    }
+
+    @Override
+    public void resetOnFailure(Exception e) throws AsterixException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public boolean alter(Map<String, String> configuration) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public InflowState setNextRecord() throws Exception {
+        boolean moreData = tweetIterator.hasNext();
+        if (!moreData) {
+            return InflowState.NO_MORE_DATA;
+        }
+        TweetMessage msg = tweetIterator.next();
+        if (isOutputFormatRecord) {
+            writeTweetRecord(msg);
+        } else {
+            writeTweetString(msg);
+        }
+        if (tweetInterval != 0) {
+            tweetCount++;
+            if (tweetCount == numTweetsBeforeDelay) {
+                Thread.sleep(tweetInterval);
+                tweetCount = 0;
+            }
+        }
+        tweetCountBeforeException++;
+
+        if (tweetCountBeforeException == exceptionPeriod) {
+            tweetCountBeforeException = 0;
+            throw new AsterixException("Delibrate exception");
+        }
+        return InflowState.DATA_AVAILABLE;
+    }
+
+    private void initialize() throws AsterixException {
+        ARecordType userRecordType = (ARecordType) outputRecordType.getFieldTypes()[1];
+        IAObject[] userMutableFields = new IAObject[] { new AMutableString(""), new AMutableString(""),
+                new AMutableInt32(0), new AMutableInt32(0), new AMutableString(""), new AMutableInt32(0) };
+        AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+        mutableFields = new IAObject[] { new AMutableString(""), new AMutableRecord(userRecordType, userMutableFields),
+                new AMutablePoint(0, 0), new AMutableDateTime(0), new AMutableUnorderedList(unorderedListType),
+                new AMutableString("") };
+        recordSerDe = new ARecordSerializerDeserializer(outputRecordType);
+        mutableRecord = new AMutableRecord(outputRecordType, mutableFields);
+
+    }
+
+    @Override
+    public void stop() {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void setOutputStream(OutputStream os) {
+        this.os = os;
+    }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java
new file mode 100644
index 0000000..6d0b1f9
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java
@@ -0,0 +1,169 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.TweetMessageIterator;
+
+public class TweetGenerator2 {
+
+    private static final Logger LOGGER = Logger.getLogger(TweetGenerator.class.getName());
+
+    public static final String KEY_DURATION = "duration";
+    public static final String KEY_TPS = "tps";
+    public static final String KEY_MIN_TPS = "tps-min";
+    public static final String KEY_MAX_TPS = "tps-max";
+
+
+    public static final String KEY_TPUT_DURATION = "tput-duration";
+  
+    public static final String OUTPUT_FORMAT = "output-format";
+
+    public static final String OUTPUT_FORMAT_ARECORD = "arecord";
+    public static final String OUTPUT_FORMAT_ADM_STRING = "adm-string";
+
+    private int duration;
+    private long tweetInterval;
+    private int numTweetsBeforeDelay;
+    private TweetMessageIterator tweetIterator = null;
+    private long exeptionInterval;
+
+  
+
+    private int partition;
+    private int tweetCount = 0;
+    private int frameTweetCount = 0;
+    private int numFlushedTweets = 0;
+
+    public int getTweetCount() {
+        return tweetCount;
+    }
+
+    private int exceptionPeriod = -1;
+    private boolean isOutputFormatRecord = false;
+    private byte[] EOL = "\n".getBytes();
+    private OutputStream os;
+    private DataGenerator2 dataGenerator = null;
+    private ByteBuffer outputBuffer = ByteBuffer.allocate(32 * 1024);
+    private int flushedTweetCount = 0;
+
+    public TweetGenerator2(Map<String, String> configuration, int partition, String format) throws Exception {
+        String value = configuration.get(KEY_DURATION);
+        duration = value != null ? Integer.parseInt(value) : 60;
+        initializeTweetRate(configuration.get(KEY_TPS));
+        if (value != null) {
+            exceptionPeriod = Integer.parseInt(value);
+        }
+       
+        isOutputFormatRecord = format.equalsIgnoreCase(OUTPUT_FORMAT_ARECORD);
+        InitializationInfo info = new InitializationInfo();
+        info.timeDurationInSecs = duration;
+        dataGenerator = new DataGenerator2(info);
+        tweetIterator = dataGenerator.new TweetMessageIterator(duration);
+    }
+
+    private void initializeTweetRate(String tps) {
+        numTweetsBeforeDelay = 0;
+        if (tps == null) {
+            tweetInterval = 0;
+        } else {
+            int val = Integer.parseInt(tps);
+            double interval = new Double(((double) 1000 / val));
+            if (interval > 1) {
+                tweetInterval = (long) interval;
+                numTweetsBeforeDelay = 1;
+            } else {
+                tweetInterval = 1;
+                Double numTweets = new Double(1 / interval);
+                if (numTweets.intValue() != numTweets) {
+                    tweetInterval = 5;
+                    numTweetsBeforeDelay = (new Double(10 * numTweets * 1)).intValue();
+                } else {
+                    numTweetsBeforeDelay = new Double((numTweets * 1)).intValue();
+                }
+            }
+        }
+
+    }
+
+    private void writeTweetString(TweetMessage next) throws IOException {
+        String tweet = next.toString() + "\n";
+        tweetCount++;
+        byte[] b = tweet.getBytes();
+        if (outputBuffer.position() + b.length > outputBuffer.limit()) {
+            flush();
+            numFlushedTweets += frameTweetCount;
+            frameTweetCount = 0;
+            flushedTweetCount += tweetCount - 1;
+            outputBuffer.put(tweet.getBytes());
+            frameTweetCount++;
+
+        } else {
+            outputBuffer.put(tweet.getBytes());
+            frameTweetCount++;
+        }
+    }
+
+    public int getNumFlushedTweets() {
+        return numFlushedTweets;
+    }
+
+    public int getFrameTweetCount() {
+        return frameTweetCount;
+    }
+
+    private void flush() throws IOException {
+        outputBuffer.flip();
+        os.write(outputBuffer.array(), 0, outputBuffer.limit());
+        outputBuffer.position(0);
+        outputBuffer.limit(32 * 1024);
+    }
+
+    private void writeTweetRecord(TweetMessage next) {
+        throw new UnsupportedOperationException("Invalid format");
+    }
+
+    public boolean setNextRecord() throws Exception {
+        boolean moreData = tweetIterator.hasNext();
+        if (!moreData) {
+            return false;
+        }
+        TweetMessage msg = tweetIterator.next();
+        if (isOutputFormatRecord) {
+            writeTweetRecord(msg);
+        } else {
+            writeTweetString(msg);
+        }
+        if (tweetInterval != 0) {
+            tweetCount++;
+            if (tweetCount == numTweetsBeforeDelay) {
+                Thread.sleep(tweetInterval);
+                tweetCount = 0;
+            }
+        }
+        return true;
+    }
+
+    public boolean setNextRecordBatch(int numTweetsInBatch) throws Exception {
+        int count = 0;
+        // if (tweetIterator.hasNext()) {
+        while (count < numTweetsInBatch) {
+            writeTweetString(tweetIterator.next());
+            count++;
+        }
+        // } else {
+        //   System.out.println("Flushing last batch, count so far:" + tweetCount);
+        // flush();
+        /// }
+        return true;
+    }
+
+    public void setOutputStream(OutputStream os) {
+        this.os = os;
+    }
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
new file mode 100644
index 0000000..5739c70
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
@@ -0,0 +1,299 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.Date;
+import java.util.Map;
+import java.util.Random;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.external.dataset.adapter.StreamBasedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+/**
+ * TPS can be configured between 1 and 20,000
+ * 
+ */
+public class TwitterFirehoseFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseFeedAdapter.class.getName());
+
+    private final TwitterServer twitterServer;
+
+    private TwitterClient twitterClient;
+
+    private static final String LOCALHOST = "127.0.0.1";
+    private static final int PORT = 2909;
+
+    private ExecutorService executorService = Executors.newCachedThreadPool();
+
+    public TwitterFirehoseFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
+            ARecordType outputtype, IHyracksTaskContext ctx) throws Exception {
+        super(parserFactory, outputtype, ctx);
+        this.twitterServer = new TwitterServer(configuration, outputtype, executorService);
+        this.twitterClient = new TwitterClient(twitterServer.getPort());
+    }
+
+    @Override
+    public void start(int partition, IFrameWriter writer) throws Exception {
+        twitterServer.start();
+        twitterServer.getListener().setPartition(partition);
+        twitterClient.start();
+        super.start(partition, writer);
+    }
+
+    @Override
+    public InputStream getInputStream(int partition) throws IOException {
+        return twitterClient.getInputStream();
+    }
+
+    private static class TwitterServer {
+        private ServerSocket serverSocket;
+        private final Listener listener;
+        private int port = -1;
+        private ExecutorService executorService;
+
+        public TwitterServer(Map<String, String> configuration, ARecordType outputtype, ExecutorService executorService)
+                throws Exception {
+            int numAttempts = 0;
+            while (port < 0) {
+                try {
+                    serverSocket = new ServerSocket(PORT + numAttempts);
+                    port = PORT + numAttempts;
+                } catch (Exception e) {
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("port: " + (PORT + numAttempts) + " unusable ");
+                    }
+                    numAttempts++;
+                }
+            }
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Twitter server configured to use port: " + port);
+            }
+            String dvds = configuration.get("dataverse-dataset");
+            listener = new Listener(serverSocket, configuration, outputtype, dvds);
+            this.executorService = executorService;
+        }
+
+        public Listener getListener() {
+            return listener;
+        }
+
+        public void start() {
+            executorService.execute(listener);
+        }
+
+        public void stop() throws IOException {
+            listener.stop();
+            serverSocket.close();
+        }
+
+        public int getPort() {
+            return port;
+        }
+
+    }
+
+    private static class TwitterClient {
+
+        private Socket socket;
+        private int port;
+
+        public TwitterClient(int port) throws UnknownHostException, IOException {
+            this.port = port;
+        }
+
+        public InputStream getInputStream() throws IOException {
+            return socket.getInputStream();
+        }
+
+        public void start() throws UnknownHostException, IOException {
+            socket = new Socket(LOCALHOST, port);
+        }
+
+    }
+
+    private static class Listener implements Runnable {
+
+        private final ServerSocket serverSocket;
+        private Socket socket;
+        private TweetGenerator2 tweetGenerator;
+        private boolean continuePush = true;
+        private int fixedTps = -1;
+        private int minTps = -1;
+        private int maxTps = -1;
+        private int tputDuration;
+        private int partition;
+        private Rate task;
+        private Mode mode;
+
+        public static final String KEY_MODE = "mode";
+
+        public static enum Mode {
+            AGGRESSIVE,
+            CONTROLLED,
+        }
+
+        public void setPartition(int partition) {
+            this.partition = partition;
+            task.setPartition(partition);
+        }
+
+        public Listener(ServerSocket serverSocket, Map<String, String> configuration, ARecordType outputtype,
+                String datasetName) throws Exception {
+            this.serverSocket = serverSocket;
+            this.tweetGenerator = new TweetGenerator2(configuration, 0, TweetGenerator.OUTPUT_FORMAT_ADM_STRING);
+            String value = configuration.get(KEY_MODE);
+            String confValue = null;
+            if (value != null) {
+                mode = Mode.valueOf(value.toUpperCase());
+                switch (mode) {
+                    case AGGRESSIVE:
+                        break;
+                    case CONTROLLED:
+                        confValue = configuration.get(TweetGenerator2.KEY_TPS);
+                        if (confValue != null) {
+                            minTps = Integer.parseInt(confValue);
+                            maxTps = minTps;
+                            fixedTps = minTps;
+                        } else {
+                            confValue = configuration.get(TweetGenerator2.KEY_MIN_TPS);
+                            if (confValue != null) {
+                                minTps = Integer.parseInt(confValue);
+                            }
+                            confValue = configuration.get(TweetGenerator2.KEY_MAX_TPS);
+                            if (confValue != null) {
+                                maxTps = Integer.parseInt(configuration.get(TweetGenerator2.KEY_MAX_TPS));
+                            }
+
+                            if (minTps < 0 || maxTps < 0 || minTps > maxTps) {
+                                throw new IllegalArgumentException("Incorrect value for min/max TPS");
+                            }
+                        }
+
+                }
+            } else {
+                mode = Mode.AGGRESSIVE;
+            }
+
+            tputDuration = Integer.parseInt(configuration.get(TweetGenerator2.KEY_TPUT_DURATION));
+            task = new Rate(tweetGenerator, tputDuration, datasetName, partition);
+
+        }
+
+        @Override
+        public void run() {
+            while (true) {
+                try {
+                    socket = serverSocket.accept();
+                    OutputStream os = socket.getOutputStream();
+                    tweetGenerator.setOutputStream(os);
+                    boolean moreData = true;
+                    Timer timer = new Timer();
+                    timer.schedule(task, tputDuration * 1000, tputDuration * 1000);
+                    long startBatch;
+                    long endBatch;
+                    Random random = new Random();
+                    int tps = 0;
+                    while (moreData && continuePush) {
+                        if(maxTps > 0){
+                             tps = minTps + random.nextInt((maxTps+1) - minTps);   
+                        } else {
+                            tps = fixedTps;
+                        }
+                        startBatch = System.currentTimeMillis();
+                        moreData = tweetGenerator.setNextRecordBatch(tps);
+                        endBatch = System.currentTimeMillis();
+                        if (mode.equals(Mode.CONTROLLED)) {
+                            if (endBatch - startBatch < 1000) {
+                                Thread.sleep(1000 - (endBatch - startBatch));
+                            }
+                        }
+                    }
+                    timer.cancel();
+                    os.close();
+                    break;
+                } catch (Exception e) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Exception in adaptor " + e.getMessage());
+                    }
+                } finally {
+                    try {
+                        if (socket != null && socket.isClosed()) {
+                            socket.close();
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info("Closed socket:" + socket.getPort());
+                            }
+                        }
+                    } catch (IOException e) {
+                        e.printStackTrace();
+                    }
+
+                }
+            }
+        }
+
+        public void stop() {
+            continuePush = false;
+        }
+
+        private static class Rate extends TimerTask {
+
+            private TweetGenerator2 gen;
+            int prevMeasuredTweets = 0;
+            private int tputDuration;
+            private int partition;
+            private String dataset;
+
+            public Rate(TweetGenerator2 gen, int tputDuration, String dataset, int partition) {
+                this.gen = gen;
+                this.tputDuration = tputDuration;
+                this.dataset = dataset;
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning(new Date() + " " + "Dataset" + " " + "partition" + " " + "Total flushed tweets"
+                            + "\t" + "intantaneous throughput");
+                }
+            }
+
+            @Override
+            public void run() {
+
+                int currentMeasureTweets = gen.getNumFlushedTweets();
+
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine(dataset + " " + partition + " " + gen.getNumFlushedTweets() + "\t"
+                            + ((currentMeasureTweets - prevMeasuredTweets) / tputDuration) + " ID "
+                            + Thread.currentThread().getId());
+                }
+
+                prevMeasuredTweets = currentMeasureTweets;
+
+            }
+
+            public void setPartition(int partition) {
+                this.partition = partition;
+            }
+        }
+    }
+
+    @Override
+    public void stop() throws Exception {
+        twitterServer.stop();
+    }
+
+}
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
new file mode 100644
index 0000000..1d8a607
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
@@ -0,0 +1,142 @@
+/*
+x * Copyright 2009-2012 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.tools.external.data;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{TwitterFirehoseFeedAdapter}.
+ * The adapter simulates a twitter firehose with tweets being "pushed" into Asterix at a configurable rate
+ * measured in terms of TPS (tweets/second). The stream of tweets lasts for a configurable duration (measured in seconds).
+ */
+public class TwitterFirehoseFeedAdapterFactory extends StreamBasedAdapterFactory implements ITypedAdapterFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    /*
+     * The dataverse and dataset names for the target feed dataset. This informaiton 
+     * is used in configuring partition constraints for the adapter. It is preferred that 
+     * the adapter location does not coincide with a partition location for the feed dataset.
+     */
+    private static final String KEY_DATAVERSE_DATASET = "dataverse-dataset";
+
+    /*
+     * Degree of parallelism for feed ingestion activity. Defaults to 1.
+     * This builds up the count constraint for the ingestion operator.
+     */
+    private static final String KEY_INGESTION_CARDINALITY = "ingestion-cardinality";
+
+    /*
+     * The absolute locations where ingestion operator instances will be places. 
+     */
+    private static final String KEY_INGESTION_LOCATIONS = "ingestion-location";
+
+    private static final ARecordType outputType = initOutputType();
+
+    @Override
+    public String getName() {
+        return "twitter_firehose";
+    }
+
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.TYPED;
+    }
+
+    @Override
+    public SupportedOperation getSupportedOperations() {
+        return SupportedOperation.READ;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        configuration.put(KEY_FORMAT, FORMAT_ADM);
+        this.configuration = configuration;
+        this.configureFormat(initOutputType());
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        String ingestionCardinalityParam = (String) configuration.get(KEY_INGESTION_CARDINALITY);
+        String ingestionLocationParam = (String) configuration.get(KEY_INGESTION_LOCATIONS);
+        String[] locations = null;
+        if (ingestionLocationParam != null) {
+            locations = ingestionLocationParam.split(",");
+        }
+        int count = locations != null ? locations.length : 1;
+        if (ingestionCardinalityParam != null) {
+            count = Integer.parseInt(ingestionCardinalityParam);
+        }
+
+        List<String> chosenLocations = new ArrayList<String>();
+        String[] availableLocations = locations != null ? locations : AsterixClusterProperties.INSTANCE
+                .getParticipantNodes().toArray(new String[] {});
+        for (int i = 0, k = 0; i < count; i++, k = (k + 1) % availableLocations.length) {
+            chosenLocations.add(availableLocations[k]);
+        }
+        return new AlgebricksAbsolutePartitionConstraint(chosenLocations.toArray(new String[] {}));
+    }
+
+    @Override
+    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+        return new TwitterFirehoseFeedAdapter(configuration, parserFactory, outputType, ctx);
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return outputType;
+    }
+
+    private static ARecordType initOutputType() {
+        ARecordType outputType = null;
+        try {
+            String[] userFieldNames = new String[] { "screen-name", "lang", "friends_count", "statuses_count", "name",
+                    "followers_count" };
+
+            IAType[] userFieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+                    BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT32 };
+            ARecordType userRecordType = new ARecordType("TwitterUserType", userFieldNames, userFieldTypes, false);
+
+            String[] fieldNames = new String[] { "tweetid", "user", "sender-location", "send-time", "referred-topics",
+                    "message-text" };
+
+            AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+            IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, userRecordType, BuiltinType.APOINT,
+                    BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
+            outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
+
+        } catch (AsterixException e) {
+            throw new IllegalStateException("Unable to initialize output type");
+        }
+        return outputType;
+    }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/resources/test.properties b/asterix-tools/src/main/resources/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 933afcd..419438f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -134,7 +134,7 @@
         appendPage.append(logRecord, appendLSN);
         appendLSN += logRecord.getLogSize();
     }
-
+    
     private void getAndInitNewPage() {
         appendPage = null;
         while (appendPage == null) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 6fb91c8..06f5399 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -43,190 +43,203 @@
  */
 public class TransactionContext implements ITransactionContext, Serializable {
 
-    private static final long serialVersionUID = -6105616785783310111L;
-    private TransactionSubsystem transactionSubsystem;
+	private static final long serialVersionUID = -6105616785783310111L;
+	private TransactionSubsystem transactionSubsystem;
 
-    //jobId is set once and read concurrently.
-    private final JobId jobId;
+	// jobId is set once and read concurrently.
+	private final JobId jobId;
 
-    //There are no concurrent writers on both firstLSN and lastLSN 
-    //since both values are updated by serialized log appenders. 
-    //But readers and writers can be different threads, 
-    //so both LSNs are atomic variables in order to be read and written atomically.
-    private AtomicLong firstLSN;
-    private AtomicLong lastLSN;
+	// There are no concurrent writers on both firstLSN and lastLSN
+	// since both values are updated by serialized log appenders.
+	// But readers and writers can be different threads,
+	// so both LSNs are atomic variables in order to be read and written
+	// atomically.
+	private AtomicLong firstLSN;
+	private AtomicLong lastLSN;
 
-    //txnState is read and written concurrently.
-    private AtomicInteger txnState;
+	// txnState is read and written concurrently.
+	private AtomicInteger txnState;
 
-    //isTimeout is read and written under the lockMgr's tableLatch
-    //Thus, no other synchronization is required separately.
-    private boolean isTimeout;
+	// isTimeout is read and written under the lockMgr's tableLatch
+	// Thus, no other synchronization is required separately.
+	private boolean isTimeout;
 
-    //isWriteTxn can be set concurrently by multiple threads. 
-    private AtomicBoolean isWriteTxn;
+	// isWriteTxn can be set concurrently by multiple threads.
+	private AtomicBoolean isWriteTxn;
 
-    //isMetadataTxn is accessed by a single thread since the metadata is not partitioned
-    private boolean isMetadataTxn;
+	// isMetadataTxn is accessed by a single thread since the metadata is not
+	// partitioned
+	private boolean isMetadataTxn;
 
-    //indexMap is concurrently accessed by multiple threads, 
-    //so those threads are synchronized on indexMap object itself
-    private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
+	// indexMap is concurrently accessed by multiple threads,
+	// so those threads are synchronized on indexMap object itself
+	private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
 
-    //TODO: fix ComponentLSNs' issues. 
-    //primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be modified accordingly
-    //when the issues of componentLSNs are fixed.  
-    private ILSMIndex primaryIndex;
-    private PrimaryIndexModificationOperationCallback primaryIndexCallback;
-    private PrimaryIndexOperationTracker primaryIndexOpTracker;
+	// TODO: fix ComponentLSNs' issues.
+	// primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be
+	// modified accordingly
+	// when the issues of componentLSNs are fixed.
+	private ILSMIndex primaryIndex;
+	private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+	private PrimaryIndexOperationTracker primaryIndexOpTracker;
 
-    //The following three variables are used as temporary variables in order to avoid object creations.
-    //Those are used in synchronized methods. 
-    private MutableLong tempResourceIdForRegister;
-    private MutableLong tempResourceIdForSetLSN;
-    private LogRecord logRecord;
+	// The following three variables are used as temporary variables in order to
+	// avoid object creations.
+	// Those are used in synchronized methods.
+	private MutableLong tempResourceIdForRegister;
+	private MutableLong tempResourceIdForSetLSN;
+	private LogRecord logRecord;
 
-    //TODO: implement transactionContext pool in order to avoid object creations.
-    //      also, the pool can throttle the number of concurrent active jobs at every moment. 
-    public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
-        this.jobId = jobId;
-        this.transactionSubsystem = transactionSubsystem;
-        firstLSN = new AtomicLong(-1);
-        lastLSN = new AtomicLong(-1);
-        txnState = new AtomicInteger(ITransactionManager.ACTIVE);
-        isTimeout = false;
-        isWriteTxn = new AtomicBoolean(false);
-        isMetadataTxn = false;
-        indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
-        primaryIndex = null;
-        tempResourceIdForRegister = new MutableLong();
-        tempResourceIdForSetLSN = new MutableLong();
-        logRecord = new LogRecord();
-    }
+	// TODO: implement transactionContext pool in order to avoid object
+	// creations.
+	// also, the pool can throttle the number of concurrent active jobs at every
+	// moment.
+	public TransactionContext(JobId jobId,
+			TransactionSubsystem transactionSubsystem) throws ACIDException {
+		this.jobId = jobId;
+		this.transactionSubsystem = transactionSubsystem;
+		firstLSN = new AtomicLong(-1);
+		lastLSN = new AtomicLong(-1);
+		txnState = new AtomicInteger(ITransactionManager.ACTIVE);
+		isTimeout = false;
+		isWriteTxn = new AtomicBoolean(false);
+		isMetadataTxn = false;
+		indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
+		primaryIndex = null;
+		tempResourceIdForRegister = new MutableLong();
+		tempResourceIdForSetLSN = new MutableLong();
+		logRecord = new LogRecord();
+	}
 
-    public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
-            boolean isPrimaryIndex) {
-        synchronized (indexMap) {
-            if (isPrimaryIndex && primaryIndex == null) {
-                primaryIndex = index;
-                primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
-                primaryIndexOpTracker = (PrimaryIndexOperationTracker) index.getOperationTracker();
-            }
-            tempResourceIdForRegister.set(resourceId);
-            if (!indexMap.containsKey(tempResourceIdForRegister)) {
-                indexMap.put(new MutableLong(resourceId),
-                        ((AbstractLSMIOOperationCallback) index.getIOOperationCallback()));
-            }
-        }
-    }
+	public void registerIndexAndCallback(long resourceId, ILSMIndex index,
+			AbstractOperationCallback callback, boolean isPrimaryIndex) {
+		synchronized (indexMap) {
+			if (isPrimaryIndex && primaryIndex == null) {
+				primaryIndex = index;
+				primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+				primaryIndexOpTracker = (PrimaryIndexOperationTracker) index
+						.getOperationTracker();
+			}
+			tempResourceIdForRegister.set(resourceId);
+			if (!indexMap.containsKey(tempResourceIdForRegister)) {
+				indexMap.put(new MutableLong(resourceId),
+						((AbstractLSMIOOperationCallback) index
+								.getIOOperationCallback()));
+			}
+		}
+	}
 
-    //[Notice] 
-    //This method is called sequentially by the LogAppender threads. 
-    //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
-    @Override
-    public void setLastLSN(long resourceId, long LSN) {
-        synchronized (indexMap) {
-            firstLSN.compareAndSet(-1, LSN);
-            lastLSN.set(Math.max(lastLSN.get(), LSN));
-            if (resourceId != -1) {
-                //Non-update log's resourceId is -1.
-                tempResourceIdForSetLSN.set(resourceId);
-                AbstractLSMIOOperationCallback ioOpCallback = indexMap.get(tempResourceIdForSetLSN);
-                ioOpCallback.updateLastLSN(LSN);
-            }
-        }
-    }
+	// [Notice]
+	// This method is called sequentially by the LogAppender threads.
+	// However, the indexMap is concurrently read and modified through this
+	// method and registerIndexAndCallback()
+	@Override
+	public void setLastLSN(long resourceId, long LSN) {
+		synchronized (indexMap) {
+			firstLSN.compareAndSet(-1, LSN);
+			lastLSN.set(Math.max(lastLSN.get(), LSN));
+			if (resourceId != -1) {
+				// Non-update log's resourceId is -1.
+				tempResourceIdForSetLSN.set(resourceId);
+				AbstractLSMIOOperationCallback ioOpCallback = indexMap
+						.get(tempResourceIdForSetLSN);
+				ioOpCallback.updateLastLSN(LSN);
+			}
+		}
+	}
 
-    @Override
-    public void notifyOptracker(boolean isJobLevelCommit) {
-        try {
-            if (isJobLevelCommit && isMetadataTxn) {
-                primaryIndexOpTracker.exclusiveJobCommitted();
-            } else if (!isJobLevelCommit) {
-                primaryIndexOpTracker
-                        .completeOperation(null, LSMOperationType.MODIFICATION, null, primaryIndexCallback);
-            }
-        } catch (HyracksDataException e) {
-            throw new IllegalStateException(e);
-        }
-    }
+	@Override
+	public void notifyOptracker(boolean isJobLevelCommit) {
+		try {
+			if (isJobLevelCommit && isMetadataTxn) {
+				primaryIndexOpTracker.exclusiveJobCommitted();
+			} else if (!isJobLevelCommit) {
+				primaryIndexOpTracker.completeOperation(null,
+						LSMOperationType.MODIFICATION, null,
+						primaryIndexCallback);
+			}
+		} catch (HyracksDataException e) {
+			throw new IllegalStateException(e);
+		}
+	}
 
-    public void setWriteTxn(boolean isWriteTxn) {
-        this.isWriteTxn.set(isWriteTxn);
-    }
+	public void setWriteTxn(boolean isWriteTxn) {
+		this.isWriteTxn.set(isWriteTxn);
+	}
 
-    public boolean isWriteTxn() {
-        return isWriteTxn.get();
-    }
+	public boolean isWriteTxn() {
+		return isWriteTxn.get();
+	}
 
-    @Override
-    public long getFirstLSN() {
-        return firstLSN.get();
-    }
+	@Override
+	public long getFirstLSN() {
+		return firstLSN.get();
+	}
 
-    @Override
-    public long getLastLSN() {
-        return lastLSN.get();
-    }
+	@Override
+	public long getLastLSN() {
+		return lastLSN.get();
+	}
 
-    public JobId getJobId() {
-        return jobId;
-    }
+	public JobId getJobId() {
+		return jobId;
+	}
 
-    public void setTimeout(boolean isTimeout) {
-        this.isTimeout = isTimeout;
-    }
+	public void setTimeout(boolean isTimeout) {
+		this.isTimeout = isTimeout;
+	}
 
-    public boolean isTimeout() {
-        return isTimeout;
-    }
+	public boolean isTimeout() {
+		return isTimeout;
+	}
 
-    public void setTxnState(int txnState) {
-        this.txnState.set(txnState);
-    }
+	public void setTxnState(int txnState) {
+		this.txnState.set(txnState);
+	}
 
-    public int getTxnState() {
-        return txnState.get();
-    }
+	public int getTxnState() {
+		return txnState.get();
+	}
 
-    @Override
-    public int hashCode() {
-        return jobId.getId();
-    }
+	@Override
+	public int hashCode() {
+		return jobId.getId();
+	}
 
-    @Override
-    public boolean equals(Object o) {
-        return (o == this);
-    }
+	@Override
+	public boolean equals(Object o) {
+		return (o == this);
+	}
 
-    @Override
-    public void setMetadataTransaction(boolean isMetadataTxn) {
-        this.isMetadataTxn = isMetadataTxn;
-    }
+	@Override
+	public void setMetadataTransaction(boolean isMetadataTxn) {
+		this.isMetadataTxn = isMetadataTxn;
+	}
 
-    @Override
-    public boolean isMetadataTransaction() {
-        return isMetadataTxn;
-    }
+	@Override
+	public boolean isMetadataTransaction() {
+		return isMetadataTxn;
+	}
 
-    public String prettyPrint() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n" + jobId + "\n");
-        sb.append("isWriteTxn: " + isWriteTxn + "\n");
-        sb.append("firstLSN: " + firstLSN.get() + "\n");
-        sb.append("lastLSN: " + lastLSN.get() + "\n");
-        sb.append("TransactionState: " + txnState + "\n");
-        sb.append("isTimeout: " + isTimeout + "\n");
-        return sb.toString();
-    }
+	public String prettyPrint() {
+		StringBuilder sb = new StringBuilder();
+		sb.append("\n" + jobId + "\n");
+		sb.append("isWriteTxn: " + isWriteTxn + "\n");
+		sb.append("firstLSN: " + firstLSN.get() + "\n");
+		sb.append("lastLSN: " + lastLSN.get() + "\n");
+		sb.append("TransactionState: " + txnState + "\n");
+		sb.append("isTimeout: " + isTimeout + "\n");
+		return sb.toString();
+	}
 
-    public LogRecord getLogRecord() {
-        return logRecord;
-    }
+	public LogRecord getLogRecord() {
+		return logRecord;
+	}
 
-    public void cleanupForAbort() {
-        if (primaryIndexOpTracker != null) {
-            primaryIndexOpTracker.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
-        }
-    }
+	public void cleanupForAbort() {
+		if (primaryIndexOpTracker != null) {
+			primaryIndexOpTracker
+					.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+		}
+	}
 }