miscellaneous/minor changes  + support for installing adapters
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 c087bd1..7a44d72 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
@@ -20,6 +20,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
@@ -29,14 +30,10 @@
 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;
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 c2c75bc..418a83e 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
@@ -30,9 +30,11 @@
 import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.feeds.FeedManager;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -89,6 +91,8 @@
     private IIOManager ioManager;
     private boolean isShuttingdown;
 
+    private IFeedManager feedManager;
+
     public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) throws AsterixException {
         this.ncApplicationContext = ncApplicationContext;
         compilerProperties = new AsterixCompilerProperties(ASTERIX_PROPERTIES_ACCESSOR);
@@ -138,6 +142,8 @@
                 txnProperties);
         isShuttingdown = false;
 
+        feedManager = new FeedManager(ncApplicationContext.getNodeId());
+
         // The order of registration is important. The buffer cache must registered before recovery and transaction managers.
         LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) bufferCache);
         LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) indexLifecycleManager);
@@ -238,4 +244,9 @@
         return metadataMergePolicyFactory;
     }
 
+    @Override
+    public IFeedManager getFeedManager() {
+        return feedManager;
+    }
+
 }
\ No newline at end of file
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
index f044ac5..5cb53b2 100644
--- 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
@@ -31,13 +31,13 @@
 import javax.servlet.http.HttpServletResponse;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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;
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
index 6985019..437a2c9 100644
--- 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
@@ -29,13 +29,12 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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;
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
index 546ff4f..96b3c31 100644
--- 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
@@ -29,12 +29,11 @@
 import javax.servlet.http.HttpServletResponse;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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;
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
index 433f834..ff29a23 100644
--- 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
@@ -8,8 +8,8 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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 {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
index d550689..472eb19 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
@@ -23,13 +23,13 @@
 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.parser.AQLParser;
+import edu.uci.ics.asterix.aql.parser.ParseException;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.asterix.aql.parser.AQLParser;
-import edu.uci.ics.asterix.aql.parser.ParseException;
 
 public class AsterixJavaClient {
     private IHyracksClientConnection hcc;
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 a18d97c..a0acd44 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
@@ -70,6 +70,7 @@
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.file.DatasetOperations;
 import edu.uci.ics.asterix.file.DataverseOperations;
@@ -98,7 +99,6 @@
 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;
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
index a6c1f8b..172a0ec 100644
--- 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
@@ -24,10 +24,8 @@
 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 {
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
index ecf1a14e..e0d7e6e 100755
--- 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
@@ -23,6 +23,8 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.Unmarshaller;
@@ -30,18 +32,23 @@
 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.external.library.ExternalLibrary;
+import edu.uci.ics.asterix.external.library.LibraryAdapter;
+import edu.uci.ics.asterix.external.library.LibraryFunction;
 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.DatasourceAdapter;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
+import edu.uci.ics.asterix.metadata.functions.ExternalLibraryManager;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 
 public class ExternalLibraryBootstrap {
 
+    private static Logger LOGGER = Logger.getLogger(ExternalLibraryBootstrap.class.getName());
+
     public static void setUpExternaLibraries(boolean isMetadataNode) throws Exception {
 
         Map<String, List<String>> uninstalledLibs = null;
@@ -108,12 +115,21 @@
             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 + ":")) {
+                if (function.getName().startsWith(libraryName + "#")) {
                     MetadataManager.INSTANCE.dropFunction(mdTxnCtx, new FunctionSignature(dataverse,
                             function.getName(), function.getArity()));
                 }
             }
 
+            List<edu.uci.ics.asterix.metadata.entities.DatasourceAdapter> adapters = MetadataManager.INSTANCE
+                    .getDataverseAdapters(mdTxnCtx, dataverse);
+            for (edu.uci.ics.asterix.metadata.entities.DatasourceAdapter adapter : adapters) {
+                if (adapter.getAdapterIdentifier().getAdapterName().startsWith(libraryName + "#")) {
+                    MetadataManager.INSTANCE.dropAdapter(mdTxnCtx, dataverse, adapter.getAdapterIdentifier()
+                            .getAdapterName());
+                }
+            }
+
             MetadataManager.INSTANCE.dropLibrary(mdTxnCtx, dataverse, libraryName);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
@@ -149,7 +165,7 @@
                 }
             });
 
-            Library library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
+            ExternalLibrary library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
 
             if (libraryDescriptors.length == 0) {
                 throw new Exception("No library descriptors defined");
@@ -162,7 +178,7 @@
                 MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverse,
                         NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, IMetadataEntity.PENDING_NO_OP));
             }
-            for (Function function : library.getFunctions().getFunction()) {
+            for (LibraryFunction function : library.getLibraryFunctions().getLibraryFunction()) {
                 String[] fargs = function.getArguments().trim().split(",");
                 List<String> args = new ArrayList<String>();
                 for (String arg : fargs) {
@@ -174,6 +190,14 @@
                 MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
             }
 
+            for (LibraryAdapter adapter : library.getLibraryAdapters().getLibraryAdapter()) {
+                String adapterFactoryClass = adapter.getFactoryClass();
+                String adapterName = libraryName + "#" + adapter.getName();
+                AdapterIdentifier aid = new AdapterIdentifier(dataverse, adapterName);
+                DatasourceAdapter dsa = new DatasourceAdapter(aid, adapterFactoryClass, AdapterType.EXTERNAL);
+                MetadataManager.INSTANCE.addAdapter(mdTxnCtx, dsa);
+            }
+
             MetadataManager.INSTANCE.addLibrary(mdTxnCtx, new edu.uci.ics.asterix.metadata.entities.Library(dataverse,
                     libraryName));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -189,17 +213,20 @@
         ExternalLibraryManager.registerLibraryClassLoader(dataverse, libraryName, classLoader);
     }
 
-    private static Library getLibrary(File libraryXMLPath) throws Exception {
-        JAXBContext configCtx = JAXBContext.newInstance(Library.class);
+    private static ExternalLibrary getLibrary(File libraryXMLPath) throws Exception {
+        JAXBContext configCtx = JAXBContext.newInstance(ExternalLibrary.class);
         Unmarshaller unmarshaller = configCtx.createUnmarshaller();
-        Library library = (Library) unmarshaller.unmarshal(libraryXMLPath);
+        ExternalLibrary library = (ExternalLibrary) 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());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Installing lirbary " + libraryName + " in dataverse " + dataverse + "."
+                    + " Install Directory: " + installDir.getAbsolutePath());
+        }
 
         File libDir = new File(installDir.getAbsolutePath() + File.separator + dataverse + File.separator + libraryName);
         FilenameFilter jarFileFilter = new FilenameFilter() {
@@ -209,8 +236,6 @@
         };
 
         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");
         }
@@ -238,6 +263,15 @@
                 urls[count++] = file.toURL();
             }
         }
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            StringBuilder logMesg = new StringBuilder("Classpath for library " + libraryName + "\n");
+            for (URL url : urls) {
+                logMesg.append(url.getFile() + "\n");
+            }
+            LOGGER.info(logMesg.toString());
+        }
+
         ClassLoader classLoader = new URLClassLoader(urls, parentClassLoader);
         return classLoader;
     }
@@ -253,48 +287,3 @@
     }
 
 }
-
-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
index 37c6e8a..f0e51b2 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
@@ -41,6 +41,8 @@
 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.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
 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;
@@ -61,7 +63,6 @@
 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;
@@ -69,7 +70,6 @@
 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;
@@ -98,6 +98,11 @@
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
 
+/**
+ * A listener that subscribes to events associated with cluster membership (nodes joining/leaving the cluster)
+ * and job lifecycle (start/end of a job). Subscription to such events allows keeping track of feed ingestion jobs
+ * and take any corrective action that may be required when a node involved in a feed leaves the cluster.
+ */
 public class FeedLifecycleListener implements IJobLifecycleListener, IClusterEventsSubscriber, Serializable {
 
     private static final long serialVersionUID = 1L;
@@ -495,7 +500,7 @@
                 if (LOGGER.isLoggable(Level.WARNING)) {
                     LOGGER.warning("Revived Feed");
                 }
-             
+
             }
         }
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
index 3197bbb..fa27e7e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
@@ -16,7 +16,6 @@
 
 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;
@@ -117,8 +116,6 @@
             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) {
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 187394a..1f8ab08 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
@@ -196,12 +196,12 @@
     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();
+            AsterixTransactionProperties txnProperties = ((IAsterixPropertiesProvider) runtimeContext)
+                    .getTransactionProperties();
             Node self = null;
             for (Node node : cluster.getSubstituteNodes().getNode()) {
                 String ncId = asterixInstanceName + "_" + node.getId();
@@ -231,7 +231,6 @@
                 throw new IllegalStateException("Unknown node joining the cluster");
             }
         }
-
     }
 
     /**
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 43cf9ec..47bf635 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
@@ -6,5 +6,4 @@
 { "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/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 17bfcee..f15c481 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
@@ -26,5 +26,4 @@
 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; 
-
+apply function feed_processor;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
index ec92972..1d4f619 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
@@ -1,6 +1,7 @@
 /*
  * Description  : Create a feed dataset that uses the synthetic feed simulator adapter. 
-                  The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed. The                   feed lasts a configured duration with data arriving at a configured rate (tweets per second). 
+                  The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed. 
+                  The feed lasts a configured duration with data arriving at a configured rate (tweets per second). 
                   Verify the existence of data after the feed finishes.
                   
  * Expected Res : Success
@@ -21,19 +22,17 @@
 } 
 
 create type TweetMessageType as closed {
-	tweetid: string,
-        user: TwitterUserType,
-        sender-location: point,
+	tweetid: int64,
+    user: TwitterUserType,
+    sender-location: point,
 	send-time: datetime,
-        referred-topics: {{ string }},
+    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"))
-
+using twitter_firehose
+(("duration"="5"),("tps"="50"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index f05efe6..26ed19b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -19,6 +19,7 @@
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -62,6 +63,8 @@
     public void deinitialize() throws HyracksDataException;
 
     public double getBloomFilterFalsePositiveRate();
-
+    
     public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
+
+    public IFeedManager getFeedManager();
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
similarity index 97%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
index a487168..def7c10 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds;
 
 import java.io.Serializable;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
similarity index 82%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
index 40b2f26..3d0d8f9 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
@@ -1,4 +1,4 @@
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -11,25 +11,29 @@
 
 /**
  * Sends feed report messages on behalf of an operator instance
- * to the SuperFeedMaanger associated with the feed.
+ * to the SuperFeedManager associated with the feed.
  */
 public class FeedMessageService {
 
     private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
+
+    public static final char MessageSeparator = '|';
     private static final char EOL = (char) "\n".getBytes()[0];
 
     private final FeedConnectionId feedId;
     private final LinkedBlockingQueue<String> inbox;
     private final FeedMessageHandler mesgHandler;
+    private final IFeedManager feedManager;
 
-    public FeedMessageService(FeedConnectionId feedId) {
+    public FeedMessageService(FeedConnectionId feedId, IFeedManager feedManager) {
         this.feedId = feedId;
-        this.inbox = new LinkedBlockingQueue<String>();
-        mesgHandler = new FeedMessageHandler(inbox, feedId);
+        inbox = new LinkedBlockingQueue<String>();
+        mesgHandler = new FeedMessageHandler(inbox, feedId, feedManager);
+        this.feedManager = feedManager;
     }
 
     public void start() throws UnknownHostException, IOException, Exception {
-        FeedManager.INSTANCE.getFeedExecutorService(feedId).execute(mesgHandler);
+        feedManager.getFeedExecutorService(feedId).execute(mesgHandler);
     }
 
     public void stop() throws IOException {
@@ -46,10 +50,12 @@
         private final FeedConnectionId feedId;
         private Socket sfmSocket;
         private boolean process = true;
+        private final IFeedManager feedManager;
 
-        public FeedMessageHandler(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId) {
+        public FeedMessageHandler(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId, IFeedManager feedManager) {
             this.inbox = inbox;
             this.feedId = feedId;
+            this.feedManager = feedManager;
         }
 
         public void run() {
@@ -66,7 +72,7 @@
                     }
                 }
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("ENDED FEED MESSAGE SERVICE for " + feedId);
+                    LOGGER.info("Ended feed message service for " + feedId);
                 }
             } catch (Exception e) {
                 if (LOGGER.isLoggable(Level.WARNING)) {
@@ -95,11 +101,11 @@
 
         private Socket obtainSFMSocket() throws UnknownHostException, IOException, Exception {
             Socket sfmDirServiceSocket = null;
-            SuperFeedManager sfm = FeedManager.INSTANCE.getSuperFeedManager(feedId);
+            SuperFeedManager sfm = feedManager.getSuperFeedManager(feedId);
             try {
-                FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+                FeedRuntimeManager runtimeManager = feedManager.getFeedRuntimeManager(feedId);
                 sfmDirServiceSocket = runtimeManager.createClientSocket(sfm.getHost(), sfm.getPort(),
-                        FeedManager.SOCKET_CONNECT_TIMEOUT);
+                        IFeedManager.SOCKET_CONNECT_TIMEOUT);
                 if (sfmDirServiceSocket == null) {
                     if (LOGGER.isLoggable(Level.WARNING)) {
                         LOGGER.warning("Unable to connect to " + sfm.getHost() + "[" + sfm.getPort() + "]");
@@ -126,7 +132,7 @@
                                 + sfm.getHost() + " " + port);
                     }
                     sfmSocket = runtimeManager.createClientSocket(sfm.getHost(), port,
-                            FeedManager.SOCKET_CONNECT_TIMEOUT);
+                            IFeedManager.SOCKET_CONNECT_TIMEOUT);
                 }
             } catch (Exception e) {
                 e.printStackTrace();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java
new file mode 100644
index 0000000..cda56ae
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java
@@ -0,0 +1,116 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.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-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
similarity index 98%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
index 29b494e..d1b2faf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds;
 
 import java.nio.ByteBuffer;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
similarity index 84%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
index b4951ec..92aa4ee 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
@@ -1,4 +1,18 @@
-package edu.uci.ics.asterix.metadata.feeds;
+/*
+ * 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.common.feeds;
 
 import java.io.IOException;
 import java.net.ServerSocket;
@@ -13,17 +27,17 @@
 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;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
 
 public class FeedRuntimeManager {
 
     private static Logger LOGGER = Logger.getLogger(FeedRuntimeManager.class.getName());
 
     private final FeedConnectionId feedId;
+    private final IFeedManager feedManager;
     private SuperFeedManager superFeedManager;
     private final Map<FeedRuntimeId, FeedRuntime> feedRuntimes;
     private final ExecutorService executorService;
@@ -31,11 +45,12 @@
     private SocketFactory socketFactory = new SocketFactory();
     private final LinkedBlockingQueue<String> feedReportQueue;
 
-    public FeedRuntimeManager(FeedConnectionId feedId) {
+    public FeedRuntimeManager(FeedConnectionId feedId, IFeedManager feedManager) {
         this.feedId = feedId;
         feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
         executorService = Executors.newCachedThreadPool();
         feedReportQueue = new LinkedBlockingQueue<String>();
+        this.feedManager = feedManager;
     }
 
     public void close(boolean closeAll) throws IOException {
@@ -44,7 +59,7 @@
         if (messageService != null) {
             messageService.stop();
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Shut down message service s for :" + feedId);
+                LOGGER.info("Shut down message services for :" + feedId);
             }
             messageService = null;
         }
@@ -73,7 +88,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Started Super Feed Manager for feed :" + feedId);
         }
-        this.messageService = new FeedMessageService(feedId);
+        this.messageService = new FeedMessageService(feedId, feedManager);
         messageService.start();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Started Feed Message Service for feed :" + feedId);
@@ -92,7 +107,7 @@
         feedRuntimes.put(runtimeId, feedRuntime);
     }
 
-    public void deregisterFeedRuntime(FeedRuntimeId runtimeId) {
+    public void deregisterFeedRuntime(FeedRuntimeId runtimeId)  {
         feedRuntimes.remove(runtimeId);
         if (feedRuntimes.isEmpty()) {
             synchronized (this) {
@@ -100,7 +115,7 @@
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("De-registering feed");
                     }
-                    FeedManager.INSTANCE.deregisterFeed(runtimeId.getFeedId());
+                    feedManager.deregisterFeed(runtimeId.getFeedId());
                 }
             }
         }
@@ -173,12 +188,6 @@
             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;
@@ -188,11 +197,6 @@
                 this.port = port;
             }
 
-            public SocketId(int port) {
-                this.host = "127.0.0.1";
-                this.port = port;
-            }
-
             public String getHost() {
                 return host;
             }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java
similarity index 61%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java
index ac33966..6cdc45c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java
@@ -12,42 +12,54 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds;
 
 import java.io.IOException;
 import java.util.concurrent.ExecutorService;
 
-import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
 
 /**
  * Handle (de)registration of feeds for delivery of control messages.
  */
 public interface IFeedManager {
 
+    public static final long SOCKET_CONNECT_TIMEOUT = 5000;
+
     /**
+     * Returns the executor service associated with the feed.
+     * 
      * @param feedId
      * @return
      */
     public ExecutorService getFeedExecutorService(FeedConnectionId feedId);
 
     /**
+     * Allows registration of a feedRuntime.
+     * 
      * @param feedRuntime
      * @throws Exception
      */
     public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception;
 
     /**
+     * Allows de-registration of a feed runtime.
+     * 
      * @param feedRuntimeId
      */
     public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId);
 
     /**
+     * Obtain feed runtime corresponding to a feedRuntimeId
+     * 
      * @param feedRuntimeId
      * @return
      */
     public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId);
 
     /**
+     * Register the Super Feed Manager associated witht a feed.
+     * 
      * @param feedId
      * @param sfm
      * @throws Exception
@@ -55,15 +67,35 @@
     public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception;
 
     /**
+     * Obtain a handle to the Super Feed Manager associated with the feed.
+     * 
      * @param feedId
      * @return
      */
     public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId);
 
     /**
+     * De-register a feed
+     * 
      * @param feedId
      * @throws IOException
      */
-    void deregisterFeed(FeedConnectionId feedId) throws IOException;
+    void deregisterFeed(FeedConnectionId feedId);
+
+    /**
+     * Obtain the feed runtime manager associated with a feed.
+     * 
+     * @param feedId
+     * @return
+     */
+    public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId);
+
+    /**
+     * Obtain a handle to the feed Message service associated with a feed.
+     * 
+     * @param feedId
+     * @return
+     */
+    public FeedMessageService getFeedMessageService(FeedConnectionId feedId);
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java
new file mode 100644
index 0000000..7beb212
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java
@@ -0,0 +1,179 @@
+/*
+ * 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.common.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/SuperFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java
similarity index 91%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java
index a45456b6..37306a0 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -29,10 +29,6 @@
 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)
@@ -85,6 +81,8 @@
 
     private boolean started = false;
 
+    private final IFeedManager feedManager;
+
     public static final int PORT_RANGE_ASSIGNED = 10;
 
     public enum FeedReportMessageType {
@@ -92,8 +90,10 @@
         THROUGHPUT
     }
 
-    public SuperFeedManager(FeedConnectionId feedId, String host, String nodeId, int port) throws Exception {
+    public SuperFeedManager(FeedConnectionId feedId, String host, String nodeId, int port, IFeedManager feedManager)
+            throws Exception {
         this.feedConnectionId = feedId;
+        this.feedManager = feedManager;
         this.nodeId = nodeId;
         this.feedReportPort = port;
         this.feedReportSubscribePort = port + 1;
@@ -128,12 +128,12 @@
 
     public void start() throws IOException {
         if (sfmService == null) {
-            ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedConnectionId);
+            ExecutorService executorService = feedManager.getFeedExecutorService(feedConnectionId);
             sfmService = new FeedReportDestinationSocketProvider(feedReportPort, feedReportInbox, feedConnectionId,
-                    availablePort);
+                    availablePort, feedManager);
             executorService.execute(sfmService);
             subscriptionService = new SuperFeedReportSubscriptionService(feedConnectionId, feedReportSubscribePort,
-                    sfmService.getMesgAnalyzer(), availablePort);
+                    sfmService.getMesgAnalyzer(), availablePort, feedManager);
             executorService.execute(subscriptionService);
         }
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -174,13 +174,15 @@
         private String EOM = "\n";
         private final FeedReportProvider reportProvider;
         private final List<FeedDataProviderService> dataProviders = new ArrayList<FeedDataProviderService>();
+        private final IFeedManager feedManager;
 
         public SuperFeedReportSubscriptionService(FeedConnectionId feedId, int port, FeedReportProvider reportProvider,
-                AtomicInteger nextPort) throws IOException {
+                AtomicInteger nextPort, IFeedManager feedManager) throws IOException {
             this.feedId = feedId;
-            serverFeedSubscribe = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+            serverFeedSubscribe = feedManager.getFeedRuntimeManager(feedId).createServerSocket(port);
             this.subscriptionPort = nextPort;
             this.reportProvider = reportProvider;
+            this.feedManager = feedManager;
         }
 
         public void stop() {
@@ -199,10 +201,10 @@
                     int port = subscriptionPort.incrementAndGet();
                     LinkedBlockingQueue<String> reportInbox = new LinkedBlockingQueue<String>();
                     reportProvider.registerSubsription(reportInbox);
-                    FeedDataProviderService dataProviderService = new FeedDataProviderService(feedId, port, reportInbox);
+                    FeedDataProviderService dataProviderService = new FeedDataProviderService(feedId, port,
+                            reportInbox, feedManager);
                     dataProviders.add(dataProviderService);
-                    FeedManager.INSTANCE.getFeedRuntimeManager(feedId).getExecutorService()
-                            .execute(dataProviderService);
+                    feedManager.getFeedRuntimeManager(feedId).getExecutorService().execute(dataProviderService);
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("Recevied subscription request for feed :" + feedId
                                 + " Subscripton available at port " + subscriptionPort);
@@ -219,18 +221,20 @@
     private static class FeedDataProviderService implements Runnable {
 
         private final FeedConnectionId feedId;
+        private final IFeedManager feedManager;
         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 {
+        public FeedDataProviderService(FeedConnectionId feedId, int port, LinkedBlockingQueue<String> inbox,
+                IFeedManager feedManager) throws IOException {
             this.feedId = feedId;
             this.subscriptionPort = port;
             this.inbox = inbox;
-            dataProviderSocket = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+            dataProviderSocket = feedManager.getFeedRuntimeManager(feedId).createServerSocket(port);
+            this.feedManager = feedManager;
         }
 
         @Override
@@ -279,15 +283,15 @@
         private boolean process = true;
 
         public FeedReportDestinationSocketProvider(int port, LinkedBlockingQueue<String> inbox,
-                FeedConnectionId feedId, AtomicInteger availablePort) throws IOException {
-            FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+                FeedConnectionId feedId, AtomicInteger availablePort, IFeedManager feedManager) throws IOException {
+            FeedRuntimeManager runtimeManager = feedManager.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);
+            feedManager.getFeedExecutorService(feedId).execute(mesgAnalyzer);
         }
 
         public void stop() {
@@ -407,7 +411,7 @@
                                     String tp = ingestionThroughputs.get(i + "");
                                     if (tp != null) {
                                         ingestionThroughputs.put(i + "", null);
-                                        finalMessage.append(tp + "|");
+                                        finalMessage.append(tp + FeedMessageService.MessageSeparator);
                                     } else {
                                         dispatchReport = false;
                                         break;
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 f81c0d5..b10ebcf 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
@@ -25,7 +25,6 @@
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 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;
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 2e68588..1a8c0df 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
@@ -116,12 +116,12 @@
 
     protected INodeResolver getNodeResolver() {
         if (nodeResolver == null) {
-            nodeResolver = initNodeResolver();
+            nodeResolver = initializeNodeResolver();
         }
         return nodeResolver;
     }
 
-    private static INodeResolver initNodeResolver() {
+    private static INodeResolver initializeNodeResolver() {
         INodeResolver nodeResolver = null;
         String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
         if (configuredNodeResolverFactory != null) {
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 c7f25b0..41f1d56 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
@@ -19,7 +19,6 @@
 import java.util.Map;
 
 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;
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
index 8d578f7..f69eb2d 100644
--- 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
@@ -1,15 +1,26 @@
+/*
+ * 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.HashMap;
 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.metadata.feeds.ConditionalPushTupleParserFactory;
 import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
 import edu.uci.ics.asterix.om.types.ARecordType;
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
deleted file mode 100644
index da49240..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java
+++ /dev/null
@@ -1,47 +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.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/PullBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
index da2264d..e2a4b76 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
@@ -32,9 +32,9 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 /**
- * Acts as an abstract class for all pull-based external data adapters.
- * Captures the common logic for obtaining bytes from an external source
- * and packing them into frames as tuples.
+ * Acts as an abstract class for all pull-based external data adapters. Captures
+ * the common logic for obtaining bytes from an external source and packing them
+ * into frames as tuples.
  */
 public abstract class PullBasedAdapter extends AbstractFeedDatasourceAdapter implements IDatasourceAdapter,
         IFeedAdapter {
@@ -45,14 +45,13 @@
     protected ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
     protected IPullBasedFeedClient pullBasedFeedClient;
     protected ARecordType adapterOutputType;
+    protected boolean continueIngestion = true;
+    protected Map<String, String> configuration;
+
     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;
 
@@ -65,10 +64,6 @@
         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());
@@ -97,17 +92,11 @@
                     case DATA_NOT_AVAILABLE:
                         break;
                 }
-                if (alterRequested) {
-                    boolean success = pullBasedFeedClient.alter(modifiedConfiguration);
-                    if (success) {
-                        configuration = modifiedConfiguration;
-                        modifiedConfiguration = null;
-                    }
-                }
+
             } catch (Exception failureException) {
                 try {
                     failureException.printStackTrace();
-                    boolean continueIngestion = policyEnforcer.handleSoftwareFailure(failureException);
+                    boolean continueIngestion = policyEnforcer.continueIngestionPostSoftwareFailure(failureException);
                     if (continueIngestion) {
                         pullBasedFeedClient.resetOnFailure(failureException);
                         tupleBuilder.reset();
@@ -124,9 +113,7 @@
 
     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())) {
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 41143d8..019d1b7 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
@@ -19,8 +19,6 @@
 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.api.context.IHyracksTaskContext;
 
 /**
@@ -54,19 +52,6 @@
         tweetClient.stop();
     }
 
-    @Override
-    public void alter(Map<String, String> properties) {
-        alterRequested = true;
-    }
-
-    public boolean isAlterRequested() {
-        return alterRequested;
-    }
-
-    public void postAlteration() {
-        alterRequested = false;
-    }
-
     public ARecordType getAdapterOutputType() {
         return recordType;
     }
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 102482d..6ace5c5 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
@@ -15,7 +15,6 @@
 package edu.uci.ics.asterix.external.dataset.adapter;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -33,8 +32,6 @@
 
     private List<String> feedURLs = new ArrayList<String>();
     private boolean isStopRequested = false;
-    private boolean isAlterRequested = false;
-    private Map<String, String> alteredParams = new HashMap<String, String>();
     private String id_prefix = "";
 
     private IPullBasedFeedClient rssFeedClient;
@@ -56,13 +53,6 @@
     }
 
     @Override
-    public void alter(Map<String, String> properties) {
-        isAlterRequested = true;
-        this.alteredParams = properties;
-        reconfigure(properties);
-    }
-
-    @Override
     public void stop() {
         isStopRequested = true;
     }
@@ -82,14 +72,6 @@
         }
     }
 
-    public boolean isAlterRequested() {
-        return isAlterRequested;
-    }
-
-    public Map<String, String> getAlteredParams() {
-        return alteredParams;
-    }
-
     @Override
     public IPullBasedFeedClient getFeedClient(int partition) throws Exception {
         if (rssFeedClient == null) {
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
index c922a5e..c4c40fd 100755
--- 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
@@ -18,6 +18,7 @@
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.functions.ExternalLibraryManager;
 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;
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
index b5888a6..b3914e2 100755
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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.common.exceptions.AsterixException;
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
index a185000..4e3f9fc 100755
--- 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
@@ -22,6 +22,7 @@
 
 public class ExternalScalarFunctionEvaluatorFactory implements ICopyEvaluatorFactory {
 
+    private static final long serialVersionUID = 1L;
     private final IExternalFunctionInfo finfo;
     private final ICopyEvaluatorFactory[] args;
 
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
index 8aed123..192cf3e 100644
--- 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
@@ -49,7 +49,6 @@
     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];
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
index a5cac66..c0b6945 100755
--- 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
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.asterix.external.library;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
@@ -98,15 +97,5 @@
         }
     }
 
-    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/resources/feed_policy/basic.pol b/asterix-external-data/src/main/resources/feed_policy/basic.pol
deleted file mode 100644
index 8aed9f8..0000000
--- a/asterix-external-data/src/main/resources/feed_policy/basic.pol
+++ /dev/null
@@ -1,8 +0,0 @@
-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
deleted file mode 100644
index efca65e..0000000
--- a/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol
+++ /dev/null
@@ -1,11 +0,0 @@
-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
deleted file mode 100644
index 0a1da08..0000000
--- a/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol
+++ /dev/null
@@ -1,17 +0,0 @@
-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
deleted file mode 100644
index 5fa507f..0000000
--- a/asterix-external-data/src/main/resources/feed_policy/twitter.pol
+++ /dev/null
@@ -1,17 +0,0 @@
-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
index 3dc4659..f58175a 100644
--- a/asterix-external-data/src/main/resources/schema/library.xsd
+++ b/asterix-external-data/src/main/resources/schema/library.xsd
@@ -1,42 +1,66 @@
 <?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">
+<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 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="factory_class" type="xs:string" />
+	<xs:element name="adaptor_type" type="xs:string" />
 
-<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>
+	<!-- definition of complex elements -->
+	<xs:element name="libraryFunction">
+		<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="libraryFunctions">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="lib:libraryFunction" maxOccurs="unbounded" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
+
+	<xs:element name="libraryAdapter">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="lib:name" />
+				<xs:element ref="lib:factory_class" />
+				<xs:element ref="lib:adaptor_type" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
+
+	<xs:element name="libraryAdapters">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="lib:libraryAdapter" maxOccurs="unbounded" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
+
+	<xs:element name="externalLibrary">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="lib:language" />
+				<xs:element ref="lib:libraryFunctions" minOccurs="0" />
+			    <xs:element ref="lib:libraryAdapters" minOccurs="0" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
 
 </xs:schema>     
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
index 7953b62..8a11757 100644
--- 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
@@ -29,16 +29,14 @@
 
     @Override
     public void deinitialize() {
-        System.out.println(" De Initialized");
+        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);
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
index 5f9be77..ce63eec 100644
--- 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
@@ -40,7 +40,7 @@
     public void evaluate(IFunctionHelper functionHelper) throws Exception {
         JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
         long sleepInterval = rand.nextInt(range);
-     //   Thread.sleep(5);
+        Thread.sleep(sleepInterval);
         functionHelper.setResult(inputRecord);
     }
 }
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
index 0a4eca6..96eba97 100644
--- 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
@@ -24,7 +24,7 @@
 
     @Override
     public void deinitialize() {
-        System.out.println(" De Initialized");
+        // nothing to do here
     }
 
     @Override
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 f78eb31..c36726b 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
@@ -20,6 +20,7 @@
 import java.util.List;
 import java.util.Map;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
@@ -34,7 +35,6 @@
 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
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 590890a..99bd06a 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
@@ -22,6 +22,7 @@
 
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
@@ -41,7 +42,6 @@
 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;
 
@@ -790,4 +790,15 @@
         return feedActivities;
     }
 
+    public List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext mdTxnCtx, String dataverse)
+            throws MetadataException {
+        List<DatasourceAdapter> dataverseAdapters;
+        try {
+            dataverseAdapters = metadataNode.getDataverseAdapters(mdTxnCtx.getJobId(), dataverse);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        return dataverseAdapters;
+    }
+
 }
\ 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 3c7e4a3..a962837 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
@@ -29,6 +29,7 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
@@ -71,7 +72,6 @@
 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;
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 1c990af..6859e45 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,7 +16,6 @@
 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;
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 f23275c..23612c7 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
@@ -19,6 +19,7 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
@@ -36,7 +37,6 @@
 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,
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 8434847..ddfd622 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
@@ -21,6 +21,7 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -38,7 +39,6 @@
 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
@@ -449,6 +449,14 @@
     public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException;
 
+    /**
+     * @param jobId
+     * @param dataverseName
+     * @param adapterName
+     * @return
+     * @throws MetadataException
+     * @throws RemoteException
+     */
     public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
             throws MetadataException, RemoteException;
 
@@ -499,7 +507,6 @@
     public void initializeFeedActivityIdFactory(JobId jobId) throws MetadataException, RemoteException;
 
     /**
-     * 
      * @param jobId
      * @param dataverse
      * @param policy
@@ -511,7 +518,6 @@
             RemoteException;
 
     /**
-     * 
      * @param jobId
      * @throws MetadataException
      * @throws RemoteException
@@ -519,7 +525,6 @@
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
 
     /**
-     * 
      * @return
      * @throws MetadataException
      * @throws 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 0b2a6c7..f2c8b5c 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
@@ -319,8 +319,7 @@
                 "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory",
                 "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory",
                 "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory",
-                "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory",
-                "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory" };
+                "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory" };
         DatasourceAdapter adapter;
         for (String adapterClassName : builtInAdapterClassNames) {
             adapter = getAdapter(adapterClassName);
@@ -333,7 +332,7 @@
             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/dataset/hints/DatasetHints.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
index 2a80963..e27a4cb 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,7 +17,6 @@
 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;
 
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 46b8268..5b24d38 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
@@ -33,6 +33,7 @@
 import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
 import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
@@ -63,7 +64,6 @@
 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;
@@ -115,7 +115,6 @@
 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.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;
@@ -165,7 +164,6 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
-    private IHyracksClientConnection hcc;
 
     private final Dataverse defaultDataverse;
     private JobId jobId;
@@ -364,13 +362,12 @@
         } catch (AlgebricksException ae) {
             throw ae;
         } catch (Exception e) {
-            e.printStackTrace();
             throw new AlgebricksException("Unable to create adapter " + e);
         }
 
         if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ) || adapterFactory
                 .getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
-            throw new AlgebricksException("external dataset adapter does not support read operation");
+            throw new AlgebricksException(" External dataset adapter does not support read operation");
         }
 
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
@@ -431,7 +428,9 @@
 
             FeedPolicy feedPolicy = (FeedPolicy) ((AqlDataSource) dataSource).getProperties().get(
                     BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
-
+            if (feedPolicy == null) {
+                throw new AlgebricksException("Feed not configured with a policy");
+            }
             feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
             feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedConnectionId(
                     feedDataSource.getDatasourceDataverse(), feedDataSource.getDatasourceName(), feedDataSource
@@ -460,21 +459,9 @@
         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);
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
index fc5d9ca..f729683 100644
--- 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
@@ -14,10 +14,10 @@
  */
 package edu.uci.ics.asterix.metadata.declared;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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;
@@ -34,6 +34,7 @@
         feed = null;
         MetadataTransactionContext ctx = null;
         try {
+            MetadataManager.INSTANCE.acquireReadLatch();
             ctx = MetadataManager.INSTANCE.beginTransaction();
             feed = MetadataManager.INSTANCE.getFeed(ctx, feedId.getDataverse(), feedId.getFeedName());
             if (feed == null) {
@@ -51,6 +52,8 @@
                 }
             }
 
+        } finally {
+            MetadataManager.INSTANCE.releaseReadLatch();
         }
     }
 
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
index 5d1704e..6a63f28 100644
--- 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
@@ -17,9 +17,7 @@
 
 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;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
index 44ad1e9..1380d9e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
@@ -19,8 +19,10 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 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;
 
@@ -40,6 +42,8 @@
 
     private IngestionRuntime ingestionRuntime;
 
+    private final IFeedManager feedManager;
+
     public enum State {
         /*
          * Indicates that data from external source will be pushed downstream for storage 
@@ -56,10 +60,11 @@
     }
 
     public AdapterRuntimeManager(FeedConnectionId feedId, IFeedAdapter feedAdapter, FeedFrameWriter writer,
-            int partition, LinkedBlockingQueue<IFeedMessage> inbox) {
+            int partition, LinkedBlockingQueue<IFeedMessage> inbox, IFeedManager feedManager) {
         this.feedId = feedId;
         this.feedAdapter = feedAdapter;
         this.partition = partition;
+        this.feedManager = feedManager;
         this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
     }
 
@@ -67,11 +72,11 @@
     public void start() throws Exception {
         state = State.ACTIVE_INGESTION;
         ingestionRuntime = new IngestionRuntime(feedId, partition, FeedRuntimeType.INGESTION, this);
-        FeedManager.INSTANCE.registerFeedRuntime(ingestionRuntime);
+        feedManager.registerFeedRuntime(ingestionRuntime);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Registered feed runtime manager for " + this.getFeedId());
         }
-        ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedId);
+        ExecutorService executorService = feedManager.getFeedExecutorService(feedId);
         executorService.execute(adapterExecutor);
     }
 
@@ -83,9 +88,9 @@
             synchronized (this) {
                 notifyAll();
             }
-        } catch (Exception e) {
-            if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("Unable to stop adapter");
+        } catch (Exception exception) {
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.severe("Unable to stop adapter " + feedAdapter + ", encountered exception " + exception);
             }
         }
     }
@@ -136,7 +141,6 @@
                 if (LOGGER.isLoggable(Level.SEVERE)) {
                     LOGGER.severe("Exception during feed ingestion " + e.getMessage());
                 }
-                e.printStackTrace();
             } finally {
                 synchronized (runtimeManager) {
                     runtimeManager.notifyAll();
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
index 3db5916..da5fdde 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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;
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
index 2dfe886..62893e2 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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;
@@ -6,7 +20,6 @@
 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;
@@ -26,8 +39,6 @@
 
     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;
@@ -47,7 +58,6 @@
     private IValueParserFactory[] valueParserFactories;
     private char delimiter;
     private final ParserType parserType;
-    private Object lock;
 
     public enum ParserType {
         ADM,
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
index fffee64..2d56658 100644
--- 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
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+
 /**
  * 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.
@@ -29,12 +31,8 @@
         this.feedId = feedId;
     }
 
-    public FeedConnectionId getFeedId() {
-        return feedId;
-    }
-
     @Override
     public String toString() {
-        return "" + feedId;
+        return MessageType.END.name() + feedId;
     }
 }
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
index db1a62a..2109425 100644
--- 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
@@ -1,25 +1,36 @@
+/*
+ * 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.atomic.AtomicInteger;
 
 public class FeedActivityIdFactory {
-	private static AtomicInteger id = new AtomicInteger();
-	private static boolean isInitialized = false;
+    private static AtomicInteger id = new AtomicInteger();
+    private static boolean isInitialized = false;
 
-	public static boolean isInitialized() {
-		return isInitialized;
-	}
+    public static boolean isInitialized() {
+        return isInitialized;
+    }
 
-	public static void initialize(int initialId) {
-		id.set(initialId);
-		isInitialized = true;
-	}
+    public static void initialize(int initialId) {
+        id.set(initialId);
+        isInitialized = true;
+    }
 
-	public static int generateFeedActivityId() {
-		return id.incrementAndGet();
-	}
+    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/FeedFrameWriter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
index 53038d2..9e76c21 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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;
@@ -10,8 +24,12 @@
 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.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedMessageService;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
+import edu.uci.ics.asterix.common.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;
@@ -29,7 +47,7 @@
     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
+    public static final long FLUSH_THRESHOLD_TIME = 5000; // 5 seconds
 
     /** Actual frame writer provided to an operator. **/
     private IFrameWriter writer;
@@ -53,18 +71,18 @@
 
     /**
      * Detects if the operator is unable to push a frame downstream
-     * within a threshold period of time. In addition, measure the
+     * within a threshold period of time. In addition, it measure the
      * throughput as observed on the output channel of the associated operator.
      */
     private HealthMonitor healthMonitor;
 
     /**
-     * Manager scheduling of tasks
+     * A Timer instance for managing scheduling of tasks.
      */
     private Timer timer;
 
     /**
-     * Provides access to the tuples in a frame. Used in collecting statistics.
+     * Provides access to the tuples in a frame. Used in collecting statistics
      */
     private FrameTupleAccessor fta;
 
@@ -79,22 +97,22 @@
         /**
          * Failure mode of operation for an operator when
          * input frames are not pushed to the downstream operator but
-         * are buffered for future retrieval.
+         * are buffered for future retrieval. This mode is adopted
+         * during failure recovery.
          */
         STORE
     }
 
     public FeedFrameWriter(IFrameWriter writer, IOperatorNodePushable nodePushable, FeedConnectionId feedId,
             FeedPolicyEnforcer policyEnforcer, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
-            FrameTupleAccessor fta) {
+            FrameTupleAccessor fta, IFeedManager feedManager) {
         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);
+            healthMonitor = new HealthMonitor(feedId, nodeId, feedRuntimeType, partition, timer, fta, feedManager);
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Statistics collection enabled for the feed " + feedId + " " + feedRuntimeType + " ["
                         + partition + "]");
@@ -130,9 +148,9 @@
                 try {
                     if (reportHealth) {
                         fta.reset(buffer);
-                        healthMonitor.notifyStart();
+                        healthMonitor.notifyStartFrameFlushActivity();
                         writer.nextFrame(buffer);
-                        healthMonitor.notifyFinish(fta.getTupleCount());
+                        healthMonitor.notifyFinishFrameFlushActivity();
                     } else {
                         writer.nextFrame(buffer);
                     }
@@ -143,55 +161,71 @@
                 }
                 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);
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Flushed old frame (from previous failed execution) : " + buf
+                                    + " on behalf of " + nodePushable.getDisplayName());
+                        }
                     }
+                    frames.clear();
                 }
-                frames.clear();
                 break;
             case STORE:
+
+                /* TODO:
+                 * Limit the in-memory space utilized during the STORE mode. The limit (expressed in bytes) 
+                 * is a parameter specified as part of the feed ingestion policy. Below is a basic implemenation
+                 * that allocates a buffer on demand.   
+                 * */
+
                 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);
+                    LOGGER.info("Stored frame for " + nodePushable.getDisplayName());
                 }
                 break;
         }
     }
 
+    /**
+     * Detects if the operator is unable to push a frame downstream
+     * within a threshold period of time. In addition, it measure the
+     * throughput as observed on the output channel of the associated operator.
+     */
     private static class HealthMonitor extends TimerTask {
 
+        private static final String EOL = "\n";
+
         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;
+        private final FeedConnectionId feedId;
+        private final String nodeId;
+        private final FeedRuntimeType feedRuntimeType;
+        private final int partition;
+        private final long period;
+        private final FrameTupleAccessor fta;
+        private final IFeedManager feedManager;
+
+        public HealthMonitor(FeedConnectionId feedId, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
+                Timer timer, FrameTupleAccessor fta, IFeedManager feedManager) {
             this.state = FramePushState.INTIALIZED;
             this.feedId = feedId;
             this.nodeId = nodeId;
             this.feedRuntimeType = feedRuntimeType;
             this.partition = partition;
-            this.period = period;
+            this.period = FLUSH_THRESHOLD_TIME;
             this.collectThroughput = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
+            this.fta = fta;
+            this.feedManager = feedManager;
         }
 
-        public void notifyStart() {
+        public void notifyStartFrameFlushActivity() {
             startTime = System.currentTimeMillis();
             state = FramePushState.WAITING_FOR_FLUSH_COMPLETION;
         }
@@ -202,55 +236,57 @@
          */
         public void reset() {
             mesgService = null;
-            collectThroughput = true;
+            collectThroughput = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
         }
 
-        public void notifyFinish(int numTuples) {
+        public void notifyFinishFrameFlushActivity() {
             state = FramePushState.WAITNG_FOR_NEXT_FRAME;
-            numTuplesInInterval.set(numTuplesInInterval.get() + numTuples);
+            numTuplesInInterval.set(numTuplesInInterval.get() + fta.getTupleCount());
         }
 
         @Override
         public void run() {
             if (state.equals(FramePushState.WAITING_FOR_FLUSH_COMPLETION)) {
                 long currentTime = System.currentTimeMillis();
-                if (currentTime - startTime > flushThresholdTime) {
+                if (currentTime - startTime > FLUSH_THRESHOLD_TIME) {
                     if (LOGGER.isLoggable(Level.SEVERE)) {
                         LOGGER.severe("Congestion reported by " + feedRuntimeType + " [" + partition + "]");
                     }
-                    sendReportToSFM(currentTime - startTime, FeedReportMessageType.CONGESTION,
+                    sendReportToSuperFeedManager(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());
+                sendReportToSuperFeedManager(instantTput, FeedReportMessageType.THROUGHPUT, System.currentTimeMillis());
             }
             numTuplesInInterval.set(0);
         }
 
-        private void sendReportToSFM(long value, SuperFeedManager.FeedReportMessageType mesgType, long timestamp) {
+        private void sendReportToSuperFeedManager(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;
+            String message = mesgType.name().toLowerCase() + FeedMessageService.MessageSeparator + feedRep
+                    + FeedMessageService.MessageSeparator + feedRuntimeType + FeedMessageService.MessageSeparator
+                    + partition + FeedMessageService.MessageSeparator + value + FeedMessageService.MessageSeparator
+                    + nodeId + FeedMessageService.MessageSeparator + timestamp + FeedMessageService.MessageSeparator
+                    + 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 + "]");
+                    LOGGER.warning("Unable to send feed report to Super Feed Manager for feed " + feedId + " "
+                            + feedRuntimeType + "[" + partition + "]");
                 }
             }
         }
 
         private void waitTillMessageServiceIsUp() {
             while (mesgService == null) {
-                mesgService = FeedManager.INSTANCE.getFeedMessageService(feedId);
+                mesgService = feedManager.getFeedMessageService(feedId);
                 if (mesgService == null) {
                     try {
                         /**
@@ -267,7 +303,8 @@
         }
 
         public void deactivate() {
-            this.cancel();
+            // cancel the timer task to avoid future execution. 
+            cancel();
             collectThroughput = false;
         }
 
@@ -295,8 +332,9 @@
         writer.fail();
         if (healthMonitor != null && !healthMonitor.feedRuntimeType.equals(FeedRuntimeType.INGESTION)) {
             healthMonitor.deactivate();
+        } else {
+            healthMonitor.reset();
         }
-        healthMonitor.reset();
     }
 
     @Override
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
index 281a62f..859ed11 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -18,8 +18,11 @@
 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.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -52,6 +55,9 @@
     /** The adaptor factory that is used to create an instance of the feed adaptor **/
     private IAdapterFactory adapterFactory;
 
+    /** The (singleton) instance of IFeedManager **/
+    private IFeedManager feedManager;
+
     public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, IAdapterFactory adapterFactory,
             ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicy) {
         super(spec, 0, 1);
@@ -67,7 +73,10 @@
             throws HyracksDataException {
         IFeedAdapter adapter = null;
         FeedRuntimeId feedRuntimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
-        IngestionRuntime ingestionRuntime = (IngestionRuntime) FeedManager.INSTANCE.getFeedRuntime(feedRuntimeId);
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
+        this.feedManager = runtimeCtx.getFeedManager();
+        IngestionRuntime ingestionRuntime = (IngestionRuntime) feedManager.getFeedRuntime(feedRuntimeId);
         try {
             if (ingestionRuntime == null) {
                 // create an instance of a feed adaptor to ingest data.
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
index 9a70ca3..3dbd2e2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -20,8 +20,13 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 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;
@@ -34,15 +39,17 @@
 
     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 final FrameTupleAccessor fta;
+    private final IFeedManager feedManager;
+
+    private FeedRuntime ingestionRuntime;
+    private IFeedAdapter adapter;
     private FeedFrameWriter feedFrameWriter;
 
     public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedAdapter adapter,
@@ -56,7 +63,9 @@
         policyEnforcer = new FeedPolicyEnforcer(feedId, feedPolicy);
         nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
         fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
-
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
+        this.feedManager = runtimeCtx.getFeedManager();
     }
 
     @Override
@@ -66,8 +75,9 @@
         try {
             if (ingestionRuntime == null) {
                 feedFrameWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
-                        FeedRuntimeType.INGESTION, partition, fta);
-                adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox);
+                        FeedRuntimeType.INGESTION, partition, fta, feedManager);
+                adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox,
+                        feedManager);
 
                 if (adapter instanceof AbstractFeedDatasourceAdapter) {
                     ((AbstractFeedDatasourceAdapter) adapter).setFeedPolicyEnforcer(policyEnforcer);
@@ -96,7 +106,7 @@
                     adapterRuntimeMgr.wait();
                 }
             }
-            FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+            feedManager.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
             feedFrameWriter.close();
         } catch (InterruptedException ie) {
             if (policyEnforcer.getFeedPolicyAccessor().continueOnHardwareFailure()) {
@@ -104,7 +114,7 @@
                     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);
+                FeedRuntimeManager runtimeMgr = feedManager.getFeedRuntimeManager(feedId);
                 try {
                     runtimeMgr.close(false);
                 } catch (IOException ioe) {
@@ -117,8 +127,8 @@
                 if (LOGGER.isLoggable(Level.INFO)) {
                     LOGGER.info("Interrupted Exception, something went wrong");
                 }
-               
-                FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+
+                feedManager.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
                 feedFrameWriter.close();
                 throw new HyracksDataException(ie);
             }
@@ -132,33 +142,3 @@
         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
index d910ca8..8b92994 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
@@ -20,21 +20,29 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedMessageService;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
 
+/**
+ * An implementation of the IFeedManager interface.
+ * Provider necessary central repository for registering/retrieving
+ * artifacts/services associated with a feed.
+ */
 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>();
+    private final String nodeId;
+
+    public FeedManager(String nodeId) {
+        this.nodeId = nodeId;
+    }
 
     public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
         return feedRuntimeManagers.get(feedId);
@@ -45,6 +53,7 @@
         return mgr == null ? null : mgr.getExecutorService();
     }
 
+    @Override
     public FeedMessageService getFeedMessageService(FeedConnectionId feedId) {
         FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
         return mgr == null ? null : mgr.getMessageService();
@@ -56,7 +65,7 @@
             FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
             if (mgr == null) {
                 if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("unknown feed id: " + feedId);
+                    LOGGER.warning("Unknown feed id: " + feedId);
                 }
             } else {
                 if (LOGGER.isLoggable(Level.INFO)) {
@@ -81,7 +90,7 @@
         if (runtimeMgr == null) {
             synchronized (feedRuntimeManagers) {
                 if (runtimeMgr == null) {
-                    runtimeMgr = new FeedRuntimeManager(feedId);
+                    runtimeMgr = new FeedRuntimeManager(feedId, this);
                     feedRuntimeManagers.put(feedId, runtimeMgr);
                 }
             }
@@ -126,4 +135,9 @@
         FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
         return runtimeMgr != null ? runtimeMgr.getSuperFeedManager() : null;
     }
+
+    @Override
+    public String toString() {
+        return "FeedManager " + "[" + nodeId + "]";
+    }
 }
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
index aedd124..f3cc6a8 100644
--- 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
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+
 /**
  * A feed control message containing the altered values for
  * adapter configuration parameters. This message is dispatched
@@ -41,7 +43,7 @@
 
     @Override
     public String toString() {
-        return host + "_" + nodeId + "[" + port + "]";
+        return MessageType.SUPER_FEED_MANAGER_ELECT.name() + " " + host + "_" + nodeId + "[" + port + "]";
     }
 
     public String getHost() {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
index 36fcf4b..019f21c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
-import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 
 /**
  * A control message that can be sent to the runtime instance of a
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
index fae2fe4..d33bf5e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
 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;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
index a8541f5..47b00dd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
@@ -17,8 +17,13 @@
 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.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -35,6 +40,7 @@
     private final IFeedMessage feedMessage;
     private final int partition;
     private final IHyracksTaskContext ctx;
+    private final IFeedManager feedManager;
 
     public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedMessage feedMessage,
             int partition, int nPartitions) {
@@ -42,6 +48,9 @@
         this.feedMessage = feedMessage;
         this.partition = partition;
         this.ctx = ctx;
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
+        this.feedManager = runtimeCtx.getFeedManager();
     }
 
     @Override
@@ -49,7 +58,7 @@
         try {
             writer.open();
             FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
-            FeedRuntime feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+            FeedRuntime feedRuntime = feedManager.getFeedRuntime(runtimeId);
             boolean ingestionLocation = feedRuntime != null;
 
             switch (feedMessage.getMessageType()) {
@@ -70,12 +79,12 @@
 
                 case SUPER_FEED_MANAGER_ELECT:
                     if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("Registering SUPER Feed MGR for :" + feedId);
+                        LOGGER.info("Registering Supers Feed Manager for :" + feedId);
                     }
                     FeedManagerElectMessage mesg = ((FeedManagerElectMessage) feedMessage);
                     SuperFeedManager sfm = new SuperFeedManager(mesg.getFeedId(), mesg.getHost(), mesg.getNodeId(),
-                            mesg.getPort());
-                    synchronized (FeedManager.INSTANCE) {
+                            mesg.getPort(), feedManager);
+                    synchronized (feedManager) {
                         INCApplicationContext ncCtx = ctx.getJobletContext().getApplicationContext();
                         String nodeId = ncCtx.getNodeId();
                         if (sfm.getNodeId().equals(nodeId)) {
@@ -83,7 +92,7 @@
                         } else {
                             Thread.sleep(5000);
                         }
-                        FeedManager.INSTANCE.registerSuperFeedManager(feedId, sfm);
+                        feedManager.registerSuperFeedManager(feedId, sfm);
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info("Registered super feed mgr " + sfm + " for feed " + feedId);
                         }
@@ -92,7 +101,6 @@
             }
 
         } 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/FeedMetaOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
index 3f8ab8b..1a8a460 100644
--- 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
@@ -4,10 +4,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeState;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.IFeedManager;
 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;
@@ -124,6 +128,9 @@
         /** Allows to iterate over the tuples in a frame **/
         private FrameTupleAccessor fta;
 
+        /** The (singleton) instance of IFeedManager **/
+        private IFeedManager feedManager;
+
         public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
                 int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
                 FeedPolicy feedPolicy, FeedRuntimeType runtimeType) throws HyracksDataException {
@@ -135,16 +142,19 @@
             this.feedId = feedConnectionId;
             this.nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
             fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+            IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                    .getApplicationContext().getApplicationObject();
+            this.feedManager = runtimeCtx.getFeedManager();
         }
 
         @Override
         public void open() throws HyracksDataException {
             FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, feedId, partition);
             try {
-                feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+                feedRuntime = feedManager.getFeedRuntime(runtimeId);
                 if (feedRuntime == null) {
                     feedRuntime = new FeedRuntime(feedId, partition, runtimeType);
-                    FeedManager.INSTANCE.registerFeedRuntime(feedRuntime);
+                    feedManager.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.");
@@ -158,11 +168,10 @@
                     resumeOldState = true;
                 }
                 FeedFrameWriter mWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
-                        runtimeType, partition, fta);
+                        runtimeType, partition, fta, feedManager);
                 coreOperatorNodePushable.setOutputFrameWriter(0, mWriter, recordDesc);
                 coreOperatorNodePushable.open();
             } catch (Exception e) {
-                e.printStackTrace();
                 if (LOGGER.isLoggable(Level.SEVERE)) {
                     LOGGER.severe("Unable to initialize feed operator " + feedRuntime + " [" + partition + "]");
                 }
@@ -182,27 +191,16 @@
                     feedRuntime.setRuntimeState(null);
                     resumeOldState = false;
                 }
+                currentBuffer = buffer;
                 coreOperatorNodePushable.nextFrame(buffer);
+                currentBuffer = null;
             } catch (HyracksDataException e) {
                 if (policyEnforcer.getFeedPolicyAccessor().continueOnApplicationFailure()) {
-                    boolean isExceptionHarmful = handleException(e.getCause());
+                    boolean isExceptionHarmful = e.getCause() instanceof TreeIndexException && !resumeOldState;
                     if (isExceptionHarmful) {
-                        // log the tuple
+                        // TODO: 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)) {
@@ -218,23 +216,6 @@
             }
         }
 
-        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)) {
@@ -248,10 +229,9 @@
                         LOGGER.warning("Saved feed compute runtime for revivals" + feedRuntime.getFeedRuntimeId());
                     }
                 } else {
-                    FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+                    feedManager.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
                     if (LOGGER.isLoggable(Level.WARNING)) {
-                        LOGGER.warning(" No state to save, de-registered feed compute runtime "
-                                + feedRuntime.getFeedRuntimeId());
+                        LOGGER.warning("No state to save, de-registered feed runtime " + feedRuntime.getFeedRuntimeId());
                     }
                 }
             }
@@ -261,7 +241,7 @@
         @Override
         public void close() throws HyracksDataException {
             coreOperatorNodePushable.close();
-            FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+            feedManager.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
         }
 
     }
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
index a3c505b..fd9716c 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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.Map;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
index b94e228..44487ec 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
@@ -1,11 +1,25 @@
+/*
+ * 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.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.common.feeds.FeedConnectionId;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.entities.FeedActivity;
@@ -24,7 +38,7 @@
                 null, new HashMap<String, String>());
     }
 
-    public boolean handleSoftwareFailure(Exception e) throws RemoteException, ACIDException {
+    public boolean continueIngestionPostSoftwareFailure(Exception e) throws RemoteException, ACIDException {
         boolean continueIngestion = feedPolicyAccessor.continueOnApplicationFailure();
         if (feedPolicyAccessor.logErrorOnFailure()) {
             persistExceptionDetails(e);
@@ -49,10 +63,6 @@
         }
     }
 
-    public void handleHardwareFailure(List<String> nodeId) {
-
-    }
-
     public FeedPolicyAccessor getFeedPolicyAccessor() {
         return feedPolicyAccessor;
     }
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
index fbff931..d3225a2 100644
--- 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
@@ -3,8 +3,9 @@
 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;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.SuperFeedManager.FeedReportMessageType;
 
 public class FeedReport implements Comparable {
 
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
deleted file mode 100644
index 268e30a..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java
+++ /dev/null
@@ -1,46 +0,0 @@
-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
index 177c082..d3693bf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
@@ -23,16 +23,19 @@
 import org.apache.commons.lang3.tuple.Pair;
 
 import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
 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.DatasourceAdapter.AdapterType;
 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.metadata.functions.ExternalLibraryManager;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -206,9 +209,24 @@
             adapterName = feed.getAdaptorName();
             adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
                     adapterName);
+            if (adapterEntity == null) {
+                adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, feed.getDataverseName(), adapterName);
+            }
+
             if (adapterEntity != null) {
                 adapterFactoryClassname = adapterEntity.getClassname();
-                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                switch (adapterEntity.getType()) {
+                    case INTERNAL:
+                        adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                        break;
+                    case EXTERNAL:
+                        String[] anameComponents = adapterName.split("#");
+                        String libraryName = anameComponents[0];
+                        ClassLoader cl = ExternalLibraryManager.getLibraryClassLoader(feed.getDataverseName(),
+                                libraryName);
+                        adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
+                        break;
+                }
             } else {
                 adapterFactoryClassname = AqlMetadataProvider.adapterFactoryMapping.get(adapterName);
                 if (adapterFactoryClassname != null) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
index 0d1f955..a03bf2b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
@@ -1,5 +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.
+ */
 package edu.uci.ics.asterix.metadata.feeds;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+
 public interface IAdapterExecutor {
 
     /**
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
index d0f8cdd..a4c5de9 100644
--- 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
@@ -30,13 +30,7 @@
      * 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.
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
index ac0c711..54613d0 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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.Map;
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
index 37d49b4..0ea0d38 100644
--- 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
@@ -14,6 +14,9 @@
  */
 package edu.uci.ics.asterix.metadata.feeds;
 
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+
 public class IngestionRuntime extends FeedRuntime {
 
     private AdapterRuntimeManager 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
index 20a6b0e..1bd042a 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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;
@@ -15,7 +29,7 @@
 
     private static final Logger LOGGER = Logger.getLogger(MessageListener.class.getName());
 
-    private int port;
+    private final int port;
     private final LinkedBlockingQueue<String> outbox;
 
     private ExecutorService executorService = Executors.newFixedThreadPool(10);
@@ -29,7 +43,9 @@
 
     public void stop() {
         listenerServer.stop();
-        System.out.println("STOPPED MESSAGE RECEIVING SERVICE AT " + port);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Stopped message service at " + port);
+        }
         if (!executorService.isShutdown()) {
             executorService.shutdownNow();
         }
@@ -37,9 +53,11 @@
     }
 
     public void start() throws IOException {
-        System.out.println("STARTING MESSAGE RECEIVING SERVICE AT " + port);
         listenerServer = new MessageListenerServer(port, outbox);
         executorService.execute(listenerServer);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Starting message service at " + port);
+        }
     }
 
     private static class MessageListenerServer implements Runnable {
@@ -107,57 +125,8 @@
 
     }
 
-    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
index 822c638..d7e2e2f 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalLibraryManager.java
similarity index 79%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalLibraryManager.java
index 520020d..aa506d0 100755
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalLibraryManager.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.library;
+package edu.uci.ics.asterix.metadata.functions;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -22,17 +22,17 @@
     private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<String, ClassLoader>();
 
     public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader) {
-        String key = dataverseName + "." + libraryName;
+        String key = getKey(dataverseName, libraryName);
         synchronized (libraryClassLoaders) {
             if (libraryClassLoaders.get(dataverseName) != null) {
-                throw new IllegalStateException("library class loader already registered!");
+                throw new IllegalStateException("Library class loader already registered!");
             }
             libraryClassLoaders.put(key, classLoader);
         }
     }
 
     public static void deregisterLibraryClassLoader(String dataverseName, String libraryName) {
-        String key = dataverseName + "." + libraryName;
+        String key = getKey(dataverseName, libraryName);
         synchronized (libraryClassLoaders) {
             if (libraryClassLoaders.get(dataverseName) != null) {
                 libraryClassLoaders.remove(key);
@@ -41,10 +41,14 @@
     }
 
     public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
-        String key = dataverseName + "." + libraryName;
+        String key = getKey(dataverseName, libraryName);
         synchronized (libraryClassLoaders) {
             return libraryClassLoaders.get(key);
         }
     }
 
+    private static String getKey(String dataverseName, String libraryName) {
+        return dataverseName + "." + libraryName;
+    }
+
 }
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
index 333d4a3..eb85bb9 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
@@ -49,8 +49,9 @@
 import edu.uci.ics.asterix.common.annotations.RecordDataGenAnnotation;
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
 import edu.uci.ics.asterix.common.annotations.UndeclaredFieldsDataGen;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -61,7 +62,6 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.types.TypeSignature;
 import edu.uci.ics.asterix.tools.translator.ADGenDmlTranslator;
-import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
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
index 0c93564..7c18670 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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.tools.external.data;
 
 import java.io.BufferedReader;
@@ -7,14 +21,11 @@
 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;
@@ -33,227 +44,50 @@
 
 public class DataGenerator {
 
-    private static RandomDateGenerator randDateGen;
-    private static RandomNameGenerator randNameGen;
-    private static RandomEmploymentGenerator randEmpGen;
-    private static RandomMessageGenerator randMessageGen;
-    private static RandomLocationGenerator randLocationGen;
+    private RandomDateGenerator randDateGen;
+    private RandomNameGenerator randNameGen;
+    private RandomEmploymentGenerator randEmpGen;
+    private RandomMessageGenerator randMessageGen;
+    private RandomLocationGenerator randLocationGen;
 
-    private static DistributionHandler fbDistHandler;
-    private static DistributionHandler twDistHandler;
+    private DistributionHandler fbDistHandler;
+    private DistributionHandler twDistHandler;
 
-    private static int totalFbMessages;
-    private static int numFbOnlyUsers;
-    private static int totalTwMessages;
-    private static int numTwOnlyUsers;
+    private int totalFbMessages;
+    private int numFbOnlyUsers;
+    private int totalTwMessages;
+    private int numTwOnlyUsers;
 
-    private static int numCommonUsers;
+    private int numCommonUsers;
 
-    private static int fbUserId;
-    private static int twUserId;
+    private int fbUserId;
+    private int twUserId;
 
-    private static int fbMessageId;
-    private static int twMessageId;
+    private int fbMessageId;
+    private int twMessageId;
 
-    private static Random random = new Random();
+    private Random random = new Random();
 
-    private static String commonUserFbSuffix = "_fb";
-    private static String commonUserTwSuffix = "_tw";
+    private String commonUserFbSuffix = "_fb";
+    private String commonUserTwSuffix = "_tw";
 
-    private static String outputDir;
+    private String outputDir;
 
-    private static PartitionConfiguration partition;
+    private PartitionConfiguration partition;
 
-    private static FacebookUser fbUser = new FacebookUser();
-    private static TwitterUser twUser = new TwitterUser();
+    private FacebookUser fbUser = new FacebookUser();
+    private TwitterUser twUser = new TwitterUser();
 
-    private static FacebookMessage fbMessage = new FacebookMessage();
-    private static TweetMessage twMessage = new TweetMessage();
+    private FacebookMessage fbMessage = new FacebookMessage();
+    private TweetMessage twMessage = new TweetMessage();
 
-    private static int duration;
+    private 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);
-        }
+    public DataGenerator(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));
@@ -292,7 +126,187 @@
         twMessageId = partition.getTargetPartition().getTwMessageIdMin();
 
         outputDir = partition.getSourcePartition().getPath();
-        generateData();
+    }
+
+    public DataGenerator(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(int partition, byte seed) {
+        return new TweetMessageIterator(duration, partition, seed);
+    }
+
+    public class TweetMessageIterator implements Iterator<TweetMessage> {
+
+        private final int duration;
+        private long startTime = 0;
+        private final GULongIDGenerator idGen;
+
+        public TweetMessageIterator(int duration, int partition, byte seed) {
+            this.duration = duration;
+            this.idGen = new GULongIDGenerator(partition, seed);
+        }
+
+        @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(idGen.getNextULong(), 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.twMessageIdMin;
+        duration = info.timeDurationInSecs;
+    }
+
+    public static void main(String args[]) throws Exception {
+        if (args.length < 2) {
+            printUsage();
+            System.exit(1);
+        }
+
+        DataGenerator dataGenerator = new DataGenerator(args);
+        dataGenerator.generateData();
     }
 
     public static void printUsage() {
@@ -300,14 +314,14 @@
         System.out.println(" Usage :" + " DataGenerator <path to configuration file> <partition name> ");
     }
 
-    public static void generateData() throws IOException {
+    public void generateData() throws IOException {
         generateFacebookOnlyUsers(numFbOnlyUsers);
         generateTwitterOnlyUsers(numTwOnlyUsers);
         generateCommonUsers();
         System.out.println("Partition :" + partition.getTargetPartition().getName() + " finished");
     }
 
-    public static void getFacebookUser(String usernameSuffix) {
+    public void getFacebookUser(String usernameSuffix) {
         String suggestedName = randNameGen.getRandomName();
         String[] nameComponents = suggestedName.split(" ");
         String name = nameComponents[0] + nameComponents[1];
@@ -322,7 +336,7 @@
         fbUser.reset(fbUserId++, alias, name, userSince, friendIds, emp);
     }
 
-    public static void getTwitterUser(String usernameSuffix) {
+    public void getTwitterUser(String usernameSuffix) {
         String suggestedName = randNameGen.getRandomName();
         String[] nameComponents = suggestedName.split(" ");
         String screenName = nameComponents[0] + nameComponents[1] + randNameGen.getRandomNameSuffix();
@@ -337,7 +351,7 @@
         twUserId++;
     }
 
-    public static void getCorrespondingTwitterUser(FacebookUser fbUser) {
+    public void getCorrespondingTwitterUser(FacebookUser fbUser) {
         String screenName = fbUser.getName().substring(0, fbUser.getName().lastIndexOf(commonUserFbSuffix))
                 + commonUserTwSuffix;
         String name = screenName.split(" ")[0];
@@ -414,16 +428,12 @@
         }
 
         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);
             }
         }
     }
@@ -730,7 +740,6 @@
     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>();
@@ -741,6 +750,7 @@
                 }
                 list.add(line);
             }
+            reader.close();
             return list;
         }
 
@@ -821,13 +831,12 @@
 
     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 Random random = new Random();
         private Point point;
 
         public RandomLocationGenerator(int beginLat, int endLat, int beginLong, int endLong) {
@@ -1266,7 +1275,7 @@
 
     public static class TweetMessage {
 
-        private String tweetid;
+        private long tweetid;
         private TwitterUser user;
         private Point senderLocation;
         private DateTime sendTime;
@@ -1277,7 +1286,7 @@
 
         }
 
-        public TweetMessage(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+        public TweetMessage(long tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
                 List<String> referredTopics, Message messageText) {
             this.tweetid = tweetid;
             this.user = user;
@@ -1287,7 +1296,7 @@
             this.messageText = messageText;
         }
 
-        public void reset(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+        public void reset(long tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
                 List<String> referredTopics, Message messageText) {
             this.tweetid = tweetid;
             this.user = user;
@@ -1301,7 +1310,7 @@
             StringBuilder builder = new StringBuilder();
             builder.append("{");
             builder.append("\"tweetid\":");
-            builder.append("\"" + tweetid + "\"");
+            builder.append("int64(\"" + tweetid + "\")");
             builder.append(",");
             builder.append("\"user\":");
             builder.append(user);
@@ -1333,11 +1342,11 @@
             return new String(builder);
         }
 
-        public String getTweetid() {
+        public long getTweetid() {
             return tweetid;
         }
 
-        public void setTweetid(String tweetid) {
+        public void setTweetid(long tweetid) {
             this.tweetid = tweetid;
         }
 
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
deleted file mode 100644
index 3fa7170..0000000
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java
+++ /dev/null
@@ -1,2487 +0,0 @@
-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(int partition, byte seed) {
-        return new TweetMessageIterator(duration, partition, seed);
-    }
-
-    public class TweetMessageIterator implements Iterator<TweetMessage> {
-
-        private final int duration;
-        private long startTime = 0;
-        private int partition;
-        private final GULongIDGenerator idGen;
-        
-        public TweetMessageIterator(int duration,int partition, byte seed) {
-            this.duration = duration;
-            this.partition = partition;
-            this.idGen = new GULongIDGenerator(partition, seed);
-        }
-
-        @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(idGen.getNextULong(), 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 long tweetid;
-        private TwitterUser user;
-        private Point senderLocation;
-        private DateTime sendTime;
-        private List<String> referredTopics;
-        private Message messageText;
-
-        public TweetMessage() {
-
-        }
-
-        public TweetMessage(long 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(long 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("int64(\"" + 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 long getTweetid() {
-            return tweetid;
-        }
-
-        public void setTweetid(long 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
deleted file mode 100644
index b2ba88e..0000000
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java
+++ /dev/null
@@ -1,18 +0,0 @@
-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/GULongIDGenerator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GULongIDGenerator.java
index 59c133c..61c4345 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GULongIDGenerator.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GULongIDGenerator.java
@@ -1,3 +1,17 @@
+/*
+ * 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.tools.external.data;
 
 import java.nio.ByteBuffer;
@@ -5,28 +19,28 @@
 
 public class GULongIDGenerator {
 
-    private final int partition;
-    private final long baseValue;
-    private final AtomicLong nextValue;
+	private final int partition;
+	private final long baseValue;
+	private final AtomicLong nextValue;
 
-    public GULongIDGenerator(int partition, byte seed) {
-        this.partition = partition;
-        ByteBuffer buffer = ByteBuffer.allocate(8);
-        buffer.put(seed);
-        buffer.put((byte) partition);
-        buffer.putInt(0);
-        buffer.putShort((short) 0);
-        buffer.flip();
-        this.baseValue = new Long(buffer.getLong());
-        this.nextValue = new AtomicLong(baseValue);
-    }
+	public GULongIDGenerator(int partition, byte seed) {
+		this.partition = partition;
+		ByteBuffer buffer = ByteBuffer.allocate(8);
+		buffer.put(seed);
+		buffer.put((byte) partition);
+		buffer.putInt(0);
+		buffer.putShort((short) 0);
+		buffer.flip();
+		this.baseValue = new Long(buffer.getLong());
+		this.nextValue = new AtomicLong(baseValue);
+	}
 
-    public long getNextULong() {
-        return nextValue.incrementAndGet();
-    }
-    
-    public int getPartition(){
-        return partition;
-    }
- 
+	public long getNextULong() {
+		return nextValue.incrementAndGet();
+	}
+
+	public int getPartition() {
+		return partition;
+	}
+
 }
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
index a536076..31c50a7 100644
--- 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
@@ -73,6 +73,9 @@
                 socket = serverSocket.accept();
                 inputStream = socket.getInputStream();
             } catch (IOException e) {
+            	if (LOGGER.isLoggable(Level.SEVERE)) {
+                    LOGGER.severe("Unable to create input stream required for feed ingestion");
+                }
                 e.printStackTrace();
             }
             return inputStream;
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
deleted file mode 100644
index d26e764..0000000
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java
+++ /dev/null
@@ -1,214 +0,0 @@
-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
deleted file mode 100644
index baa7a63..0000000
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
-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, int partition) 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
index 9d47de2..b92c3fd 100644
--- 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
@@ -1,213 +1,100 @@
+/*
+ * 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.tools.external.data;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.util.List;
+import java.nio.ByteBuffer;
 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 class TweetGenerator {
 
     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 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 KEY_GUID_SEED = "guid-seed";
 
+    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 int frameTweetCount = 0;
+    private int numFlushedTweets = 0;
     private OutputStream os;
+    private DataGenerator dataGenerator = null;
+    private ByteBuffer outputBuffer = ByteBuffer.allocate(32 * 1024);
 
-    public TweetGenerator(Map<String, String> configuration, ARecordType outputRecordType, int partition, String format)
-            throws AsterixException {
-        this.outputRecordType = outputRecordType;
+    public TweetGenerator(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));
-        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();
-    }
+        dataGenerator = new DataGenerator(info);
 
-    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();
-                }
-            }
-        }
-
+        String seedValue = configuration.get(KEY_GUID_SEED);
+        int seedInt = seedValue != null ? Integer.parseInt(seedValue) : 0;
+        tweetIterator = dataGenerator.new TweetMessageIterator(duration, partition, (byte) seedInt);
     }
 
     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);
+        String tweet = next.toString() + "\n";
+        byte[] b = tweet.getBytes();
+        if (outputBuffer.position() + b.length > outputBuffer.limit()) {
+            flush();
+            numFlushedTweets += frameTweetCount;
+            frameTweetCount = 0;
+            outputBuffer.put(b);
+            frameTweetCount++;
         } else {
-            writeTweetString(msg);
+            outputBuffer.put(b);
+            frameTweetCount++;
         }
-        if (tweetInterval != 0) {
-            tweetCount++;
-            if (tweetCount == numTweetsBeforeDelay) {
-                Thread.sleep(tweetInterval);
-                tweetCount = 0;
+    }
+
+    public int getNumFlushedTweets() {
+        return numFlushedTweets;
+    }
+
+    private void flush() throws IOException {
+        outputBuffer.flip();
+        os.write(outputBuffer.array(), 0, outputBuffer.limit());
+        outputBuffer.position(0);
+        outputBuffer.limit(32 * 1024);
+    }
+
+    public boolean setNextRecordBatch(int numTweetsInBatch) throws Exception {
+        int count = 0;
+        if (tweetIterator.hasNext()) {
+            while (count < numTweetsInBatch) {
+                writeTweetString(tweetIterator.next());
+                count++;
             }
+            return true;
         }
-        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
-
+        return false;
     }
 
     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
deleted file mode 100644
index b4db589..0000000
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java
+++ /dev/null
@@ -1,166 +0,0 @@
-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 KEY_GUID_SEED = "guid-seed";
-
-    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;
-    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 int getTweetCount() {
-        return tweetCount;
-    }
-
-    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);
-
-        String seedValue = configuration.get(KEY_GUID_SEED);
-        int seedInt = seedValue != null ? Integer.parseInt(seedValue) : 0;
-        tweetIterator = dataGenerator.new TweetMessageIterator(duration, partition, (byte) seedInt);
-    }
-
-    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
index be09685..07e018a 100644
--- 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
@@ -1,3 +1,17 @@
+/*
+ * 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.tools.external.data;
 
 import java.io.IOException;
@@ -25,7 +39,6 @@
 
 /**
  * TPS can be configured between 1 and 20,000
- * 
  */
 public class TwitterFirehoseFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
 
@@ -39,6 +52,7 @@
 
     private static final String LOCALHOST = "127.0.0.1";
     private static final int PORT = 2909;
+    private static final int TPUT_DURATION_DEFAULT = 5; // 5 seconds
 
     private ExecutorService executorService = Executors.newCachedThreadPool();
 
@@ -52,7 +66,6 @@
     @Override
     public void start(int partition, IFrameWriter writer) throws Exception {
         twitterServer.start();
-        twitterServer.getListener().setPartition(partition);
         twitterClient.start();
         super.start(partition, writer);
     }
@@ -68,8 +81,8 @@
         private int port = -1;
         private ExecutorService executorService;
 
-        public TwitterServer(Map<String, String> configuration, ARecordType outputtype, ExecutorService executorService, int partition)
-                throws Exception {
+        public TwitterServer(Map<String, String> configuration, ARecordType outputtype,
+                ExecutorService executorService, int partition) throws Exception {
             int numAttempts = 0;
             while (port < 0) {
                 try {
@@ -90,10 +103,6 @@
             this.executorService = executorService;
         }
 
-        public Listener getListener() {
-            return listener;
-        }
-
         public void start() {
             executorService.execute(listener);
         }
@@ -106,7 +115,6 @@
         public int getPort() {
             return port;
         }
-
     }
 
     private static class TwitterClient {
@@ -132,13 +140,12 @@
 
         private final ServerSocket serverSocket;
         private Socket socket;
-        private TweetGenerator2 tweetGenerator;
+        private TweetGenerator 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;
 
@@ -149,15 +156,10 @@
             CONTROLLED,
         }
 
-        public void setPartition(int partition) {
-            this.partition = partition;
-            task.setPartition(partition);
-        }
-
         public Listener(ServerSocket serverSocket, Map<String, String> configuration, ARecordType outputtype,
                 String datasetName, int partition) throws Exception {
             this.serverSocket = serverSocket;
-            this.tweetGenerator = new TweetGenerator2(configuration, partition, TweetGenerator.OUTPUT_FORMAT_ADM_STRING);
+            this.tweetGenerator = new TweetGenerator(configuration, partition, TweetGenerator.OUTPUT_FORMAT_ADM_STRING);
             String value = configuration.get(KEY_MODE);
             String confValue = null;
             if (value != null) {
@@ -166,19 +168,19 @@
                     case AGGRESSIVE:
                         break;
                     case CONTROLLED:
-                        confValue = configuration.get(TweetGenerator2.KEY_TPS);
+                        confValue = configuration.get(TweetGenerator.KEY_TPS);
                         if (confValue != null) {
                             minTps = Integer.parseInt(confValue);
                             maxTps = minTps;
                             fixedTps = minTps;
                         } else {
-                            confValue = configuration.get(TweetGenerator2.KEY_MIN_TPS);
+                            confValue = configuration.get(TweetGenerator.KEY_MIN_TPS);
                             if (confValue != null) {
                                 minTps = Integer.parseInt(confValue);
                             }
-                            confValue = configuration.get(TweetGenerator2.KEY_MAX_TPS);
+                            confValue = configuration.get(TweetGenerator.KEY_MAX_TPS);
                             if (confValue != null) {
-                                maxTps = Integer.parseInt(configuration.get(TweetGenerator2.KEY_MAX_TPS));
+                                maxTps = Integer.parseInt(configuration.get(TweetGenerator.KEY_MAX_TPS));
                             }
 
                             if (minTps < 0 || maxTps < 0 || minTps > maxTps) {
@@ -191,9 +193,9 @@
                 mode = Mode.AGGRESSIVE;
             }
 
-            tputDuration = Integer.parseInt(configuration.get(TweetGenerator2.KEY_TPUT_DURATION));
+            value = configuration.get(TweetGenerator.KEY_TPUT_DURATION);
+            tputDuration = value != null ? Integer.parseInt(value) : TPUT_DURATION_DEFAULT;
             task = new Rate(tweetGenerator, tputDuration, datasetName, partition);
-
         }
 
         @Override
@@ -209,20 +211,25 @@
                     long startBatch;
                     long endBatch;
                     Random random = new Random();
-                    int tps = 0;
+                    int batchSize = 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));
-                            }
+                        switch (mode) {
+                            case CONTROLLED:
+                                if (maxTps > 0) {
+                                    batchSize = minTps + random.nextInt((maxTps + 1) - minTps);
+                                } else {
+                                    batchSize = fixedTps;
+                                }
+                                startBatch = System.currentTimeMillis();
+                                moreData = tweetGenerator.setNextRecordBatch(batchSize);
+                                endBatch = System.currentTimeMillis();
+                                if (endBatch - startBatch < 1000) {
+                                    Thread.sleep(1000 - (endBatch - startBatch));
+                                }
+                                break;
+                            case AGGRESSIVE:
+                                batchSize = Integer.MAX_VALUE;
+                                moreData = tweetGenerator.setNextRecordBatch(batchSize);
                         }
                     }
                     timer.cancel();
@@ -254,16 +261,17 @@
 
         private static class Rate extends TimerTask {
 
-            private TweetGenerator2 gen;
-            int prevMeasuredTweets = 0;
-            private int tputDuration;
-            private int partition;
-            private String dataset;
+            private final TweetGenerator gen;
+            private final int tputDuration;
+            private final int partition;
+            private final String dataset;
+            private int prevMeasuredTweets = 0;
 
-            public Rate(TweetGenerator2 gen, int tputDuration, String dataset, int partition) {
+            public Rate(TweetGenerator gen, int tputDuration, String dataset, int partition) {
                 this.gen = gen;
                 this.tputDuration = tputDuration;
                 this.dataset = dataset;
+                this.partition = partition;
                 if (LOGGER.isLoggable(Level.WARNING)) {
                     LOGGER.warning(new Date() + " " + "Dataset" + " " + "partition" + " " + "Total flushed tweets"
                             + "\t" + "intantaneous throughput");
@@ -272,22 +280,15 @@
 
             @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;
-            }
         }
     }
 
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
index b18a5f2..2305c32 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
@@ -17,7 +17,6 @@
 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;
@@ -29,7 +28,6 @@
 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;