First stage of external data cleanup

In this change, different parts of external data were refactored.
The goal was to make it more modular, easier to maintain and allow
higher flexibility for extension in addition to reducing code redundancy.

Change-Id: I04a8c4e494d8d1363992b6fe0bdbe6b2b3b7b767
Reviewed-on: https://asterix-gerrit.ics.uci.edu/566
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <hubailmor@gmail.com>
diff --git a/asterix-algebra/pom.xml b/asterix-algebra/pom.xml
index 3e76d43..bf38393 100644
--- a/asterix-algebra/pom.xml
+++ b/asterix-algebra/pom.xml
@@ -112,7 +112,6 @@
 			<groupId>org.apache.asterix</groupId>
 			<artifactId>asterix-external-data</artifactId>
 			<version>0.8.8-SNAPSHOT</version>
-			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.asterix</groupId>
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index 0360529..9b87d9f 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -187,7 +187,6 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
-            <type>jar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
@@ -196,6 +195,12 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <type>jar</type>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
             <groupId>xerces</groupId>
             <artifactId>xercesImpl</artifactId>
             <version>2.9.1</version>
@@ -233,5 +238,10 @@
             <version>0.8.8-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-external-data</artifactId>
+            <version>0.8.8-SNAPSHOT</version>
+        </dependency>
     </dependencies>
 </project>
diff --git a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
index 08b92e7..7c055e2 100644
--- a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
@@ -63,7 +63,7 @@
 import org.apache.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.feeds.CentralFeedManager;
 import org.apache.asterix.feeds.FeedJoint;
@@ -238,13 +238,12 @@
 
     /**
      * Compiles and submits for execution a list of AQL statements.
-     *
      * @param hcc
-     *            A Hyracks client connection that is used to submit a jobspec to Hyracks.
+     *        A Hyracks client connection that is used to submit a jobspec to Hyracks.
      * @param hdc
-     *            A Hyracks dataset client object that is used to read the results.
+     *        A Hyracks dataset client object that is used to read the results.
      * @param resultDelivery
-     *            True if the results should be read asynchronously or false if we should wait for results to be read.
+     *        True if the results should be read asynchronously or false if we should wait for results to be read.
      * @return A List<QueryResult> containing a QueryResult instance corresponding to each submitted query.
      * @throws Exception
      */
@@ -2133,7 +2132,7 @@
                 FeedId feedId = connectionRequest.getFeedJointKey().getFeedId();
                 PrimaryFeed primaryFeed = (PrimaryFeed) MetadataManager.INSTANCE.getFeed(mdTxnCtx,
                         feedId.getDataverse(), feedId.getFeedName());
-                Pair<JobSpecification, IFeedAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
+                Pair<JobSpecification, IAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
                         metadataProvider, policyAccessor);
                 // adapter configuration are valid at this stage
                 // register the feed joints (these are auto-de-registered)
@@ -2141,11 +2140,13 @@
                     FeedLifecycleListener.INSTANCE.registerFeedJoint(fj);
                 }
                 JobUtils.runJob(hcc, pair.first, false);
-                IFeedAdapterFactory adapterFactory = pair.second;
+                /* TODO: Fix record tracking
+                 * IFeedAdapterFactory adapterFactory = pair.second;
                 if (adapterFactory.isRecordTrackingEnabled()) {
                     FeedLifecycleListener.INSTANCE.registerFeedIntakeProgressTracker(feedConnId,
                             adapterFactory.createIntakeProgressTracker());
                 }
+                */
                 eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_INTAKE_STARTED);
             } else {
                 for (IFeedJoint fj : triple.third) {
@@ -2186,7 +2187,6 @@
     /**
      * Generates a subscription request corresponding to a connect feed request. In addition, provides a boolean
      * flag indicating if feed intake job needs to be started (source primary feed not found to be active).
-     *
      * @param dataverse
      * @param feed
      * @param dataset
diff --git a/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java b/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
index d3fade5..595fe4c 100644
--- a/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
+++ b/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
@@ -27,11 +27,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
 import org.apache.asterix.common.api.ILocalResourceMetadata;
 import org.apache.asterix.common.config.AsterixStorageProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -45,16 +40,17 @@
 import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
 import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import org.apache.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
-import org.apache.asterix.external.adapter.factory.HiveAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
-import org.apache.asterix.external.indexing.operators.ExternalDatasetIndexesCommitOperatorDescriptor;
-import org.apache.asterix.external.indexing.operators.ExternalDatasetIndexesRecoverOperatorDescriptor;
-import org.apache.asterix.external.indexing.operators.IndexInfoOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesCommitOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesRecoverOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalFilesIndexOperatorDescriptor;
+import org.apache.asterix.external.operators.IndexInfoOperatorDescriptor;
+import org.apache.asterix.external.provider.AdapterFactoryProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -73,13 +69,17 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.AsterixAppContextInfo;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.tools.external.data.ExternalFilesIndexOperatorDescriptor;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.resource.ExternalBTreeLocalResourceMetadata;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import org.apache.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import org.apache.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -107,6 +107,7 @@
 
     public static final List<List<String>> FILE_INDEX_FIELD_NAMES = new ArrayList<List<String>>();
     public static final ArrayList<IAType> FILE_INDEX_FIELD_TYPES = new ArrayList<IAType>();
+
     static {
         FILE_INDEX_FIELD_NAMES.add(new ArrayList<String>(Arrays.asList("")));
         FILE_INDEX_FIELD_TYPES.add(BuiltinType.ASTRING);
@@ -114,9 +115,7 @@
 
     public static boolean isIndexible(ExternalDatasetDetails ds) {
         String adapter = ds.getAdapter();
-        if (adapter.equalsIgnoreCase("hdfs") || adapter.equalsIgnoreCase("hive")
-                || adapter.equalsIgnoreCase("org.apache.asterix.external.dataset.adapter.HDFSAdapter")
-                || adapter.equalsIgnoreCase("org.apache.asterix.external.dataset.adapter.HIVEAdapter")) {
+        if (adapter.equalsIgnoreCase(ExternalDataConstants.ALIAS_HDFS_ADAPTER)) {
             return true;
         }
         return false;
@@ -126,12 +125,6 @@
         return ds.getState() != ExternalDatasetTransactionState.COMMIT;
     }
 
-    public static boolean datasetUsesHiveAdapter(ExternalDatasetDetails ds) {
-        String adapter = ds.getAdapter();
-        return (adapter.equalsIgnoreCase("hive") || adapter
-                .equalsIgnoreCase("org.apache.asterix.external.dataset.adapter.HIVEAdapter"));
-    }
-
     public static boolean isValidIndexName(String datasetName, String indexName) {
         return (!datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX).equals(indexName));
     }
@@ -154,18 +147,15 @@
         return IndexingConstants.getBuddyBtreeComparatorFactories();
     }
 
-    public static ArrayList<ExternalFile> getSnapshotFromExternalFileSystem(Dataset dataset) throws AlgebricksException {
+    public static ArrayList<ExternalFile> getSnapshotFromExternalFileSystem(Dataset dataset)
+            throws AlgebricksException {
         ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
         ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
         try {
             // Create the file system object
             FileSystem fs = getFileSystemObject(datasetDetails.getProperties());
-            // If dataset uses hive adapter, add path to the dataset properties
-            if (datasetUsesHiveAdapter(datasetDetails)) {
-                HiveAdapterFactory.populateConfiguration(datasetDetails.getProperties());
-            }
             // Get paths of dataset
-            String path = datasetDetails.getProperties().get(HDFSAdapterFactory.KEY_PATH);
+            String path = datasetDetails.getProperties().get(ExternalDataConstants.KEY_PATH);
             String[] paths = path.split(",");
 
             // Add fileStatuses to files
@@ -176,9 +166,9 @@
                     if (fileStatuses[i].isDirectory()) {
                         listSubFiles(dataset, fs, fileStatuses[i], files);
                     } else {
-                        files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(),
-                                nextFileNumber, fileStatuses[i].getPath().toUri().getPath(), new Date(fileStatuses[i]
-                                        .getModificationTime()), fileStatuses[i].getLen(),
+                        files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), nextFileNumber,
+                                fileStatuses[i].getPath().toUri().getPath(),
+                                new Date(fileStatuses[i].getModificationTime()), fileStatuses[i].getLen(),
                                 ExternalFilePendingOp.PENDING_NO_OP));
                     }
                 }
@@ -216,14 +206,14 @@
 
     public static FileSystem getFileSystemObject(Map<String, String> map) throws IOException {
         Configuration conf = new Configuration();
-        conf.set("fs.default.name", map.get(HDFSAdapterFactory.KEY_HDFS_URL).trim());
-        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, map.get(ExternalDataConstants.KEY_HDFS_URL).trim());
+        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_CLASS, DistributedFileSystem.class.getName());
         return FileSystem.get(conf);
     }
 
     public static JobSpecification buildFilesIndexReplicationJobSpec(Dataset dataset,
             ArrayList<ExternalFile> externalFilesSnapshot, AqlMetadataProvider metadataProvider, boolean createIndex)
-            throws MetadataException, AlgebricksException {
+                    throws MetadataException, AlgebricksException {
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
         IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
         AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
@@ -232,20 +222,20 @@
         ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
         Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
-                        dataset.getDatasetName(), getFilesIndexName(dataset.getDatasetName()), true);
+                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(), dataset.getDatasetName(),
+                        getFilesIndexName(dataset.getDatasetName()), true);
         IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
         FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
         ILocalResourceMetadata localResourceMetadata = new ExternalBTreeLocalResourceMetadata(
-                filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                filesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, false, dataset.getDatasetId(),
-                mergePolicyFactory, mergePolicyFactoryProperties);
+                filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS, filesIndexDescription.FILES_INDEX_COMP_FACTORIES,
+                new int[] { 0 }, false, dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
         PersistentLocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.ExternalBTreeResource);
         ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
-                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
+                mergePolicyFactory, mergePolicyFactoryProperties,
+                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                storageProperties.getBloomFilterFalsePositiveRate(),
                 ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
         ExternalFilesIndexOperatorDescriptor externalFilesOp = new ExternalFilesIndexOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
@@ -260,7 +250,6 @@
 
     /**
      * This method create an indexing operator that index records in HDFS
-     * 
      * @param jobSpec
      * @param itemType
      * @param dataset
@@ -272,10 +261,10 @@
     private static Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> getExternalDataIndexingOperator(
             JobSpecification jobSpec, IAType itemType, Dataset dataset, List<ExternalFile> files,
             RecordDescriptor indexerDesc, AqlMetadataProvider metadataProvider) throws Exception {
-        HDFSIndexingAdapterFactory adapterFactory = new HDFSIndexingAdapterFactory();
-        adapterFactory.setFiles(files);
-        adapterFactory.configure(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties(),
-                (ARecordType) itemType);
+        ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+        Map<String, String> configuration = externalDatasetDetails.getProperties();
+        IAdapterFactory adapterFactory = AdapterFactoryProvider.getAdapterFactory(externalDatasetDetails.getAdapter(),
+                configuration, (ARecordType) itemType, files, true);
         return new Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint>(
                 new ExternalDataScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
                 adapterFactory.getPartitionConstraint());
@@ -297,7 +286,6 @@
      * deleteedFiles should contain files that are no longer there in the file system
      * appendedFiles should have the new file information of existing files
      * The method should return false in case of zero delta
-     * 
      * @param dataset
      * @param metadataFiles
      * @param addedFiles
@@ -309,7 +297,7 @@
      */
     public static boolean isDatasetUptodate(Dataset dataset, List<ExternalFile> metadataFiles,
             List<ExternalFile> addedFiles, List<ExternalFile> deletedFiles, List<ExternalFile> appendedFiles)
-            throws MetadataException, AlgebricksException {
+                    throws MetadataException, AlgebricksException {
         boolean uptodate = true;
         int newFileNumber = metadataFiles.get(metadataFiles.size() - 1).getFileNumber() + 1;
 
@@ -340,9 +328,10 @@
                     } else {
                         // Same file name, Different file mod date -> delete and add
                         metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
-                        deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile
-                                .getDatasetName(), 0, metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
-                                metadataFile.getSize(), ExternalFilePendingOp.PENDING_DROP_OP));
+                        deletedFiles
+                                .add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(), 0,
+                                        metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
+                                        metadataFile.getSize(), ExternalFilePendingOp.PENDING_DROP_OP));
                         fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_ADD_OP);
                         fileSystemFile.setFileNumber(newFileNumber);
                         addedFiles.add(fileSystemFile);
@@ -382,8 +371,8 @@
             if (metadataFile.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP) {
                 metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
                 deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(),
-                        newFileNumber, metadataFile.getFileName(), metadataFile.getLastModefiedTime(), metadataFile
-                                .getSize(), metadataFile.getPendingOp()));
+                        newFileNumber, metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
+                        metadataFile.getSize(), metadataFile.getPendingOp()));
                 newFileNumber++;
                 uptodate = false;
             }
@@ -421,13 +410,14 @@
                 metadataProvider.getMetadataTxnContext());
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                        dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
+                splitsAndConstraint.first,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        compactionInfo.first, compactionInfo.second,
                         new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate(), false, null, null, null, null, !temp));
-        AlgebricksPartitionConstraintHelper
-                .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
+                splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
 
         return spec;
@@ -443,9 +433,9 @@
             else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_APPEND_OP) {
                 for (ExternalFile appendedFile : appendedFiles) {
                     if (appendedFile.getFileName().equals(file.getFileName())) {
-                        files.add(new ExternalFile(file.getDataverseName(), file.getDatasetName(),
-                                file.getFileNumber(), file.getFileName(), file.getLastModefiedTime(), appendedFile
-                                        .getSize(), ExternalFilePendingOp.PENDING_NO_OP));
+                        files.add(new ExternalFile(file.getDataverseName(), file.getDatasetName(), file.getFileNumber(),
+                                file.getFileName(), file.getLastModefiedTime(), appendedFile.getSize(),
+                                ExternalFilePendingOp.PENDING_NO_OP));
                     }
                 }
             }
@@ -557,8 +547,7 @@
             AsterixStorageProperties storageProperties, JobSpecification spec) {
         return new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
                 new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
-                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
                 storageProperties.getBloomFilterFalsePositiveRate(), new int[] { index.getKeyFieldNames().size() },
                 ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds), true);
     }
@@ -567,7 +556,7 @@
     private static ExternalRTreeDataflowHelperFactory getRTreeDataflowHelperFactory(Dataset ds, Index index,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
             AsterixStorageProperties storageProperties, AqlMetadataProvider metadataProvider, JobSpecification spec)
-            throws AlgebricksException, AsterixException {
+                    throws AlgebricksException, AsterixException {
         int numPrimaryKeys = getRIDSize(ds);
         List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
         secondaryKeyFields.size();
@@ -594,8 +583,8 @@
                     .getSerializerDeserializer(nestedKeyType);
             secondaryRecFields[i] = keySerde;
 
-            secondaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                    nestedKeyType, true);
+            secondaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
+                    .getBinaryComparatorFactory(nestedKeyType, true);
             secondaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
             valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
         }
@@ -743,13 +732,14 @@
         ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
         Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
-                        dataset.getDatasetName(), getFilesIndexName(dataset.getDatasetName()), true);
+                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(), dataset.getDatasetName(),
+                        getFilesIndexName(dataset.getDatasetName()), true);
         IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
         ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
-                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
+                mergePolicyFactory, mergePolicyFactoryProperties,
+                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                storageProperties.getBloomFilterFalsePositiveRate(),
                 ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
         FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
         LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
diff --git a/asterix-app/src/main/java/org/apache/asterix/file/FeedOperations.java b/asterix-app/src/main/java/org/apache/asterix/file/FeedOperations.java
index c7bb0e3..cb55c5f 100644
--- a/asterix-app/src/main/java/org/apache/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/org/apache/asterix/file/FeedOperations.java
@@ -33,7 +33,7 @@
 import org.apache.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
 import org.apache.asterix.common.feeds.message.EndFeedMessage;
 import org.apache.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.feeds.FeedLifecycleListener;
 import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.metadata.entities.PrimaryFeed;
@@ -58,23 +58,22 @@
 
     /**
      * Builds the job spec for ingesting a (primary) feed from its external source via the feed adaptor.
-     * 
      * @param primaryFeed
      * @param metadataProvider
      * @return JobSpecification the Hyracks job specification for receiving data from external source
      * @throws Exception
      */
-    public static Pair<JobSpecification, IFeedAdapterFactory> buildFeedIntakeJobSpec(PrimaryFeed primaryFeed,
+    public static Pair<JobSpecification, IAdapterFactory> buildFeedIntakeJobSpec(PrimaryFeed primaryFeed,
             AqlMetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
 
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
         spec.setFrameSize(FeedConstants.JobConstants.DEFAULT_FRAME_SIZE);
-        IFeedAdapterFactory adapterFactory = null;
+        IAdapterFactory adapterFactory = null;
         IOperatorDescriptor feedIngestor;
         AlgebricksPartitionConstraint ingesterPc;
 
         try {
-            Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory> t = metadataProvider
+            Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t = metadataProvider
                     .buildFeedIntakeRuntime(spec, primaryFeed, policyAccessor);
             feedIngestor = t.first;
             ingesterPc = t.second;
@@ -90,7 +89,7 @@
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, ingesterPc);
         spec.connect(new OneToOneConnectorDescriptor(spec), feedIngestor, 0, nullSink, 0);
         spec.addRoot(nullSink);
-        return new Pair<JobSpecification, IFeedAdapterFactory>(spec, adapterFactory);
+        return new Pair<JobSpecification, IAdapterFactory>(spec, adapterFactory);
     }
 
     public static JobSpecification buildDiscontinueFeedSourceSpec(AqlMetadataProvider metadataProvider, FeedId feedId)
@@ -247,7 +246,7 @@
     private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
             JobSpecification jobSpec, FeedConnectionId feedConenctionId, List<String> locations,
             FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, FeedId sourceFeedId)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         IFeedMessage feedMessage = new EndFeedMessage(feedConenctionId, sourceFeedRuntimeType, sourceFeedId,
                 completeDisconnection, EndFeedMessage.EndMessageType.DISCONNECT_FEED);
         return buildSendFeedMessageRuntime(jobSpec, feedConenctionId, feedMessage, locations);
diff --git a/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java b/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java
index 4eb6944..44af0ff 100644
--- a/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java
+++ b/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java
@@ -38,7 +38,7 @@
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.operators.ExternalIndexBulkModifyOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalIndexBulkModifyOperatorDescriptor;
 import org.apache.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
index 267be3d..01775ab 100755
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
@@ -46,6 +46,7 @@
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.DatasourceAdapter.AdapterType;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.feeds.AdapterIdentifier;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
@@ -221,8 +222,7 @@
                 LOGGER.info("Installed adapters contain in library :" + libraryName);
             }
 
-            MetadataManager.INSTANCE.addLibrary(mdTxnCtx,
-                    new org.apache.asterix.metadata.entities.Library(dataverse, libraryName));
+            MetadataManager.INSTANCE.addLibrary(mdTxnCtx, new Library(dataverse, libraryName));
 
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Added library " + libraryName + "to Metadata");
diff --git a/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index 2f8a910..9eee818 100644
--- a/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
@@ -80,7 +80,7 @@
         AsterixHyracksIntegrationUtil.init(true);
         // Set the node resolver to be the identity resolver that expects node names
         // to be node controller ids; a valid assumption in test environment.
-        System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
+        System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
     }
 
diff --git a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
index 7a55c90..922486f 100644
--- a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
@@ -23,18 +23,14 @@
 import java.util.Collection;
 import java.util.logging.Logger;
 
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.config.AsterixTransactionProperties;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.asterix.testframework.xml.TestGroup;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.control.nc.application.NCApplicationContext;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
-import org.junit.*;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
diff --git a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index b072e55..5e76ecb 100644
--- a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -26,7 +26,7 @@
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.testframework.xml.TestGroup;
 import org.apache.asterix.testframework.xml.TestSuite;
@@ -64,7 +64,7 @@
         // Set the node resolver to be the identity resolver that expects node
         // names
         // to be node controller ids; a valid assumption in test environment.
-        System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
+        System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
 
         FailedGroup = new TestGroup();
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 8a23d78..c4dde05 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
@@ -1,12 +1,3 @@
-{ "DataverseName": "Metadata", "Name": "azure_twitter", "Classname": "org.apache.asterix.external.adapter.factory.PullBasedAzureTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Thu Oct 24 01:39:27 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "org.apache.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "file_feed", "Classname": "org.apache.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "org.apache.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "org.apache.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "org.apache.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "org.apache.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "push_twitter", "Classname": "org.apache.asterix.external.adapter.factory.PushBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "org.apache.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "socket_adapter", "Classname": "org.apache.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
-{ "DataverseName": "Metadata", "Name": "socket_client", "Classname": "org.apache.asterix.tools.external.data.SocketClientAdapterFactory", "Type": "INTERNAL", "Timestamp": "Wed Nov 20 14:45:58 IST 2013" }
-{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "org.apache.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "adapter", "Classname": "org.apache.asterix.external.adapter.factory.GenericAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Jan 03 15:39:35 AST 2016" }
+{ "DataverseName": "Metadata", "Name": "socket_adapter", "Classname": "org.apache.asterix.external.runtime.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Jan 03 15:39:35 AST 2016" }
+{ "DataverseName": "Metadata", "Name": "socket_client", "Classname": "org.apache.asterix.external.runtime.SocketClientAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Jan 03 15:39:35 AST 2016" }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
index cd829e7..b8c3f2f 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -40,7 +40,6 @@
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
-import org.apache.hyracks.storage.common.file.ResourceIdFactory;
 
 public interface IAsterixAppRuntimeContext {
 
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/BasicMonitoredBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/BasicMonitoredBuffer.java
index c5594db..70833fc 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/BasicMonitoredBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/BasicMonitoredBuffer.java
@@ -21,6 +21,8 @@
 import org.apache.asterix.common.feeds.api.IExceptionHandler;
 import org.apache.asterix.common.feeds.api.IFeedMetricCollector;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback;
+import org.apache.asterix.common.feeds.api.IFramePostProcessor;
+import org.apache.asterix.common.feeds.api.IFramePreprocessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/ComputeSideMonitoredBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/ComputeSideMonitoredBuffer.java
index 4ae288d..7ec3fdf 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/ComputeSideMonitoredBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/ComputeSideMonitoredBuffer.java
@@ -21,6 +21,8 @@
 import org.apache.asterix.common.feeds.api.IExceptionHandler;
 import org.apache.asterix.common.feeds.api.IFeedMetricCollector;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback;
+import org.apache.asterix.common.feeds.api.IFramePostProcessor;
+import org.apache.asterix.common.feeds.api.IFramePreprocessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IntakeSideMonitoredBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/IntakeSideMonitoredBuffer.java
index ed1e943..10b7ddb 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IntakeSideMonitoredBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/IntakeSideMonitoredBuffer.java
@@ -21,6 +21,8 @@
 import org.apache.asterix.common.feeds.api.IExceptionHandler;
 import org.apache.asterix.common.feeds.api.IFeedMetricCollector;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback;
+import org.apache.asterix.common.feeds.api.IFramePostProcessor;
+import org.apache.asterix.common.feeds.api.IFramePreprocessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/MonitoredBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/MonitoredBuffer.java
index 5761944..e5a22b5 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/MonitoredBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/MonitoredBuffer.java
@@ -35,6 +35,8 @@
 import org.apache.asterix.common.feeds.api.IFeedRuntime.Mode;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback.FrameEvent;
+import org.apache.asterix.common.feeds.api.IFramePostProcessor;
+import org.apache.asterix.common.feeds.api.IFramePreprocessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/StorageSideMonitoredBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/StorageSideMonitoredBuffer.java
index d545b09..4027237 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/StorageSideMonitoredBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/StorageSideMonitoredBuffer.java
@@ -24,6 +24,8 @@
 import org.apache.asterix.common.feeds.api.IExceptionHandler;
 import org.apache.asterix.common.feeds.api.IFeedMetricCollector;
 import org.apache.asterix.common.feeds.api.IFrameEventCallback;
+import org.apache.asterix.common.feeds.api.IFramePostProcessor;
+import org.apache.asterix.common.feeds.api.IFramePreprocessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IAdapterRuntimeManager.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IAdapterRuntimeManager.java
index 86e0a73..2eb6caa 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IAdapterRuntimeManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IAdapterRuntimeManager.java
@@ -45,14 +45,12 @@
 
     /**
      * Start feed ingestion
-     * 
      * @throws Exception
      */
     public void start() throws Exception;
 
     /**
      * Stop feed ingestion.
-     * 
      * @throws Exception
      */
     public void stop() throws Exception;
@@ -65,7 +63,7 @@
     /**
      * @return the instance of the feed adapter (an implementation of {@code IFeedAdapter}) in use.
      */
-    public IFeedAdapter getFeedAdapter();
+    public IDataSourceAdapter getFeedAdapter();
 
     /**
      * @return state associated with the AdapterRuntimeManager. See {@code State}.
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDatasourceAdapter.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDataSourceAdapter.java
similarity index 84%
rename from asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDatasourceAdapter.java
rename to asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDataSourceAdapter.java
index c1ee0f7..9dd4e76 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDatasourceAdapter.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IDataSourceAdapter.java
@@ -28,7 +28,7 @@
  * to be implemented by each adapter irrespective of the the kind of
  * adapter(pull or push).
  */
-public interface IDatasourceAdapter extends Serializable {
+public interface IDataSourceAdapter extends Serializable {
 
     /**
      * Triggers the adapter to begin ingesting data from the external source.
@@ -44,4 +44,17 @@
      * @throws Exception
      */
     public void start(int partition, IFrameWriter writer) throws Exception;
+
+    /**
+     * Discontinue the ingestion of data.
+     *
+     * @throws Exception
+     */
+    public boolean stop() throws Exception;
+
+    /**
+     * @param e
+     * @return true if the ingestion should continue post the exception else false
+     */
+    public boolean handleException(Throwable e);
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFeedAdapter.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFeedAdapter.java
deleted file mode 100644
index 2307285..0000000
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFeedAdapter.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.feeds.api;
-
-/**
- * Interface implemented by a feed adapter.
- */
-public interface IFeedAdapter extends IDatasourceAdapter {
-
-    public enum DataExchangeMode {
-        /**
-         * PULL model requires the adaptor to make a separate request each time to receive data
-         **/
-        PULL,
-
-        /**
-         * PUSH mode involves the use o just one initial request (handshake) by the adaptor
-         * to the datasource for setting up the connection and providing any protocol-specific
-         * parameters. Once a connection is established, the data source "pushes" data to the adaptor.
-         **/
-        PUSH
-    }
-
-    /**
-     * Returns the data exchange mode (PULL/PUSH) associated with the flow.
-     * 
-     * @return
-     */
-    public DataExchangeMode getDataExchangeMode();
-
-    /**
-     * Discontinue the ingestion of data and end the feed.
-     * 
-     * @throws Exception
-     */
-    public void stop() throws Exception;
-
-    /**
-     * @param e
-     * @return true if the feed ingestion should continue post the exception else false
-     */
-    public boolean handleException(Exception e);
-
-}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePostProcessor.java
similarity index 95%
rename from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
rename to asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePostProcessor.java
index 1dfbee9..ed74037 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePostProcessor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.common.feeds.api;
 
 import java.nio.ByteBuffer;
 
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePreprocessor.java
similarity index 95%
rename from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
rename to asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePreprocessor.java
index f602656..59a6c97 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/IFramePreprocessor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.common.feeds.api;
 
 import java.nio.ByteBuffer;
 
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java
index 822390a..4067508 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.common.feeds.api;
 
-public interface ITupleTrackingFeedAdapter extends IFeedAdapter {
+public interface ITupleTrackingFeedAdapter extends IDataSourceAdapter {
 
     public void tuplePersistedTimeCallback(long timestamp);
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/parse/IAsterixTupleParser.java b/asterix-common/src/main/java/org/apache/asterix/common/parse/IAsterixTupleParser.java
deleted file mode 100644
index 87f4c58..0000000
--- a/asterix-common/src/main/java/org/apache/asterix/common/parse/IAsterixTupleParser.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.parse;
-
-import java.util.Map;
-
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-
-public interface IAsterixTupleParser extends ITupleParser{
-
-    public void configure(Map<String, String> configuration);
-    
-}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwardPolicy.java b/asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwarder.java
similarity index 88%
rename from asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwardPolicy.java
rename to asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwarder.java
index df5a983..5ee065a 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwardPolicy.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/parse/ITupleForwarder.java
@@ -25,11 +25,11 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
-public interface ITupleForwardPolicy {
+public interface ITupleForwarder {
 
-    public static final String PARSER_POLICY = "parser-policy";
-    
-    public enum TupleForwardPolicyType {
+    public static final String FORWARD_POLICY = "forward-policy";
+
+    public enum TupleForwardPolicy {
         FRAME_FULL,
         COUNTER_TIMER_EXPIRED,
         RATE_CONTROLLED
@@ -39,8 +39,6 @@
 
     public void initialize(IHyracksCommonContext ctx, IFrameWriter frameWriter) throws HyracksDataException;
 
-    public TupleForwardPolicyType getType();
-
     public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException;
 
     public void close() throws HyracksDataException;
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index d8147b6..6afe692 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -29,7 +29,6 @@
 import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.net.URL;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -63,12 +62,12 @@
     private static final long MAX_URL_LENGTH = 2000l;
     private static Method managixExecuteMethod = null;
 
-    private static String host;
-    private static int port;
+    private String host;
+    private int port;
 
     public TestExecutor() {
-        this.host = "127.0.0.1";
-        this.port = 19002;
+        host = "127.0.0.1";
+        port = 19002;
     }
 
     public TestExecutor(String host, int port) {
diff --git a/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java b/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
index 96fb6ec..a10b5ea 100644
--- a/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
+++ b/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
@@ -29,6 +29,10 @@
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.event.error.EventException;
+import org.apache.asterix.event.model.AsterixInstance;
+import org.apache.asterix.installer.schema.conf.Configuration;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -38,10 +42,6 @@
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
-import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.event.error.EventException;
-import org.apache.asterix.event.model.AsterixInstance;
-import org.apache.asterix.installer.schema.conf.Configuration;
 
 public class ZooKeeperService implements ILookupService {
 
@@ -63,6 +63,7 @@
     private LinkedBlockingQueue<String> msgQ = new LinkedBlockingQueue<String>();
     private ZooKeeperWatcher watcher = new ZooKeeperWatcher(msgQ);
 
+    @Override
     public boolean isRunning(Configuration conf) throws Exception {
         List<String> servers = conf.getZookeeper().getServers().getServer();
         int clientPort = conf.getZookeeper().getClientPort().intValue();
@@ -92,6 +93,7 @@
         return isRunning;
     }
 
+    @Override
     public void startService(Configuration conf) throws Exception {
         if (LOGGER.isDebugEnabled()) {
             LOGGER.debug("Starting ZooKeeper at " + zkConnectionString);
@@ -107,22 +109,29 @@
         for (String zkServer : zkServers) {
             cmdBuffer.append(zkServer + " ");
         }
-        Runtime.getRuntime().exec(cmdBuffer.toString());
+        //TODO: Create a better way to interact with zookeeper
+        Process zkProcess = Runtime.getRuntime().exec(cmdBuffer.toString());
+        int output = zkProcess.waitFor();
+        if (output != 0) {
+            throw new Exception("Error starting zookeeper server. output code = " + output);
+        }
         zk = new ZooKeeper(zkConnectionString, ZOOKEEPER_SESSION_TIME_OUT, watcher);
-        String head = msgQ.poll(10, TimeUnit.SECONDS);
+        String head = msgQ.poll(60, TimeUnit.SECONDS);
         if (head == null) {
             StringBuilder msg = new StringBuilder(
                     "Unable to start Zookeeper Service. This could be because of the following reasons.\n");
             msg.append("1) Managix is incorrectly configured. Please run " + "managix validate"
                     + " to run a validation test and correct the errors reported.");
-            msg.append("\n2) If validation in (1) is successful, ensure that java_home parameter is set correctly in Managix configuration ("
-                    + AsterixEventServiceUtil.MANAGIX_CONF_XML + ")");
+            msg.append(
+                    "\n2) If validation in (1) is successful, ensure that java_home parameter is set correctly in Managix configuration ("
+                            + AsterixEventServiceUtil.MANAGIX_CONF_XML + ")");
             throw new Exception(msg.toString());
         }
         msgQ.take();
         createRootIfNotExist();
     }
 
+    @Override
     public void stopService(Configuration conf) throws Exception {
         if (LOGGER.isDebugEnabled()) {
             LOGGER.debug("Stopping ZooKeeper running at " + zkConnectionString);
@@ -141,6 +150,7 @@
         }
     }
 
+    @Override
     public void writeAsterixInstance(AsterixInstance asterixInstance) throws Exception {
         String instanceBasePath = ASTERIX_INSTANCE_BASE_PATH + File.separator + asterixInstance.getName();
         ByteArrayOutputStream b = new ByteArrayOutputStream();
@@ -166,6 +176,7 @@
         }
     }
 
+    @Override
     public AsterixInstance getAsterixInstance(String name) throws Exception {
         String path = ASTERIX_INSTANCE_BASE_PATH + File.separator + name;
         Stat stat = zk.exists(ASTERIX_INSTANCE_BASE_PATH + File.separator + name, false);
@@ -176,10 +187,12 @@
         return readAsterixInstanceObject(asterixInstanceBytes);
     }
 
+    @Override
     public boolean exists(String path) throws Exception {
         return zk.exists(ASTERIX_INSTANCE_BASE_PATH + File.separator + path, false) != null;
     }
 
+    @Override
     public void removeAsterixInstance(String name) throws Exception {
         if (!exists(name)) {
             throw new EventException("Asterix instance by name " + name + " does not exists.");
@@ -195,6 +208,7 @@
         zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name, DEFAULT_NODE_VERSION);
     }
 
+    @Override
     public List<AsterixInstance> getAsterixInstances() throws Exception {
         List<String> instanceNames = zk.getChildren(ASTERIX_INSTANCE_BASE_PATH, false);
         List<AsterixInstance> asterixInstances = new ArrayList<AsterixInstance>();
@@ -207,13 +221,14 @@
         return asterixInstances;
     }
 
-    private AsterixInstance readAsterixInstanceObject(byte[] asterixInstanceBytes) throws IOException,
-            ClassNotFoundException {
+    private AsterixInstance readAsterixInstanceObject(byte[] asterixInstanceBytes)
+            throws IOException, ClassNotFoundException {
         ByteArrayInputStream b = new ByteArrayInputStream(asterixInstanceBytes);
         ObjectInputStream ois = new ObjectInputStream(b);
         return (AsterixInstance) ois.readObject();
     }
 
+    @Override
     public void updateAsterixInstance(AsterixInstance updatedInstance) throws Exception {
         removeAsterixInstance(updatedInstance.getName());
         writeAsterixInstance(updatedInstance);
@@ -249,6 +264,7 @@
         this.msgQ = msgQ;
     }
 
+    @Override
     public void process(WatchedEvent wEvent) {
         if (wEvent.getState() == KeeperState.SyncConnected) {
             msgQ.add("connected");
@@ -276,7 +292,8 @@
         List<String> servers = conf.getZookeeper().getServers().getServer();
         int serverId = 1;
         for (String server : servers) {
-            buffer.append("server" + "." + serverId + "=" + server + ":" + leaderConnPort + ":" + leaderElecPort + "\n");
+            buffer.append(
+                    "server" + "." + serverId + "=" + server + ":" + leaderConnPort + ":" + leaderElecPort + "\n");
             serverId++;
         }
         AsterixEventServiceUtil.dumpToFile(zooKeeperConfigPath, buffer.toString());
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index 4062b23..867c96b 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -35,6 +35,43 @@
     <build>
         <plugins>
             <plugin>
+                <groupId>org.apache.asterix</groupId>
+                <artifactId>lexer-generator-maven-plugin</artifactId>
+                <version>0.8.8-SNAPSHOT</version>
+                <configuration>
+                    <grammarFile>src/main/resources/adm.grammar</grammarFile>
+                    <outputDir>${project.build.directory}/generated-sources/org/apache/asterix/runtime/operators/file/adm</outputDir>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>generate-lexer</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>generate-lexer</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.9</version>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources/</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
                 <groupId>org.jvnet.jaxb2.maven2</groupId>
                 <artifactId>maven-jaxb2-plugin</artifactId>
                 <version>0.9.0</version>
@@ -91,6 +128,50 @@
                 </executions>
             </plugin>
         </plugins>
+        <pluginManagement>
+            <plugins>
+                <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+                <plugin>
+                    <groupId>org.eclipse.m2e</groupId>
+                    <artifactId>lifecycle-mapping</artifactId>
+                    <version>1.0.0</version>
+                    <configuration>
+                        <lifecycleMappingMetadata>
+                            <pluginExecutions>
+                                <pluginExecution>
+                                    <pluginExecutionFilter>
+                                        <groupId> org.apache.asterix</groupId>
+                                        <artifactId> lexer-generator-maven-plugin</artifactId>
+                                        <versionRange>[0.1,)</versionRange>
+                                        <goals>
+                                            <goal>generate-lexer</goal>
+                                        </goals>
+                                    </pluginExecutionFilter>
+                                    <action>
+                                        <execute>
+                                            <runOnIncremental>false</runOnIncremental>
+                                        </execute>
+                                    </action>
+                                </pluginExecution>
+                                <pluginExecution>
+                                    <pluginExecutionFilter>
+                                        <groupId> org.codehaus.mojo</groupId>
+                                        <artifactId>build-helper-maven-plugin</artifactId>
+                                        <versionRange>[1.7,)</versionRange>
+                                        <goals>
+                                            <goal>add-source</goal>
+                                        </goals>
+                                    </pluginExecutionFilter>
+                                    <action>
+                                        <ignore />
+                                    </action>
+                                </pluginExecution>
+                            </pluginExecutions>
+                        </lifecycleMappingMetadata>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
     </build>
     <dependencies>
         <dependency>
@@ -139,6 +220,10 @@
             <scope>compile</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+        </dependency>
+        <dependency>
             <groupId>net.java.dev.rome</groupId>
             <artifactId>rome-fetcher</artifactId>
             <version>1.0.0</version>
@@ -186,5 +271,11 @@
             <artifactId>jdo2-api</artifactId>
             <version>2.3-20090302111651</version>
         </dependency>
+        <dependency>
+            <groupId>com.e-movimento.tinytools</groupId>
+            <artifactId>privilegedaccessor</artifactId>
+            <version>1.2.2</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/CNNFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
deleted file mode 100644
index 8b7b6d5..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.dataset.adapter.RSSFeedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * A factory class for creating the @see {CNNFeedAdapter}.
- */
-public class CNNFeedAdapterFactory implements IFeedAdapterFactory {
-    private static final long serialVersionUID = 1L;
-
-    private Map<String, String> configuration;
-
-    private List<String> feedURLs = new ArrayList<String>();
-    private static Map<String, String> topicFeeds = new HashMap<String, String>();
-    private ARecordType recordType;
-    public static final String KEY_RSS_URL = "topic";
-    public static final String KEY_INTERVAL = "interval";
-    public static final String TOP_STORIES = "topstories";
-    public static final String WORLD = "world";
-    public static final String US = "us";
-    public static final String SPORTS = "sports";
-    public static final String BUSINESS = "business";
-    public static final String POLITICS = "politics";
-    public static final String CRIME = "crime";
-    public static final String TECHNOLOGY = "technology";
-    public static final String HEALTH = "health";
-    public static final String ENTERNTAINMENT = "entertainemnt";
-    public static final String TRAVEL = "travel";
-    public static final String LIVING = "living";
-    public static final String VIDEO = "video";
-    public static final String STUDENT = "student";
-    public static final String POPULAR = "popular";
-    public static final String RECENT = "recent";
-
-    private void initTopics() {
-        topicFeeds.put(TOP_STORIES, "http://rss.cnn.com/rss/cnn_topstories.rss");
-        topicFeeds.put(WORLD, "http://rss.cnn.com/rss/cnn_world.rss");
-        topicFeeds.put(US, "http://rss.cnn.com/rss/cnn_us.rss");
-        topicFeeds.put(SPORTS, "http://rss.cnn.com/rss/si_topstories.rss");
-        topicFeeds.put(BUSINESS, "http://rss.cnn.com/rss/money_latest.rss");
-        topicFeeds.put(POLITICS, "http://rss.cnn.com/rss/cnn_allpolitics.rss");
-        topicFeeds.put(CRIME, "http://rss.cnn.com/rss/cnn_crime.rss");
-        topicFeeds.put(TECHNOLOGY, "http://rss.cnn.com/rss/cnn_tech.rss");
-        topicFeeds.put(HEALTH, "http://rss.cnn.com/rss/cnn_health.rss");
-        topicFeeds.put(ENTERNTAINMENT, "http://rss.cnn.com/rss/cnn_showbiz.rss");
-        topicFeeds.put(LIVING, "http://rss.cnn.com/rss/cnn_living.rss");
-        topicFeeds.put(VIDEO, "http://rss.cnn.com/rss/cnn_freevideo.rss");
-        topicFeeds.put(TRAVEL, "http://rss.cnn.com/rss/cnn_travel.rss");
-        topicFeeds.put(STUDENT, "http://rss.cnn.com/rss/cnn_studentnews.rss");
-        topicFeeds.put(POPULAR, "http://rss.cnn.com/rss/cnn_mostpopular.rss");
-        topicFeeds.put(RECENT, "http://rss.cnn.com/rss/cnn_latest.rss");
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        RSSFeedAdapter cnnFeedAdapter = new RSSFeedAdapter(configuration, recordType, ctx);
-        return cnnFeedAdapter;
-    }
-
-    @Override
-    public String getName() {
-        return "cnn_feed";
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.configuration = configuration;
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
-        if (rssURLProperty == null) {
-            throw new IllegalArgumentException("no rss url provided");
-        }
-        initializeFeedURLs(rssURLProperty);
-        this.recordType = outputType;
-    }
-
-    private void initializeFeedURLs(String rssURLProperty) {
-        feedURLs.clear();
-        String[] rssTopics = rssURLProperty.split(",");
-        initTopics();
-        for (String topic : rssTopics) {
-            String feedURL = topicFeeds.get(topic);
-            if (feedURL == null) {
-                throw new IllegalArgumentException(
-                        " unknown topic :" + topic + " please choose from the following " + getValidTopics());
-            }
-            feedURLs.add(feedURL);
-        }
-    }
-
-    private static String getValidTopics() {
-        StringBuilder builder = new StringBuilder();
-        for (String key : topicFeeds.keySet()) {
-            builder.append(key);
-            builder.append(" ");
-        }
-        return new String(builder);
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(feedURLs.size());
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return recordType;
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
new file mode 100644
index 0000000..2e7158d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.adapter.factory;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.api.IExternalDataSourceFactory;
+import org.apache.asterix.external.api.IIndexibleExternalDataSource;
+import org.apache.asterix.external.api.IIndexingAdapterFactory;
+import org.apache.asterix.external.dataset.adapter.GenericAdapter;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.provider.DataflowControllerProvider;
+import org.apache.asterix.external.provider.DatasourceFactoryProvider;
+import org.apache.asterix.external.provider.ParserFactoryProvider;
+import org.apache.asterix.external.util.ExternalDataCompatibilityUtils;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class GenericAdapterFactory implements IIndexingAdapterFactory, IAdapterFactory {
+
+    private static final long serialVersionUID = 1L;
+    private IExternalDataSourceFactory dataSourceFactory;
+    private IDataParserFactory dataParserFactory;
+    private ARecordType recordType;
+    private Map<String, String> configuration;
+    private List<ExternalFile> files;
+    private boolean indexingOp;
+
+    @Override
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) {
+        this.files = files;
+        this.indexingOp = indexingOp;
+    }
+
+    @Override
+    public String getAlias() {
+        return ExternalDataConstants.ALIAS_GENERIC_ADAPTER;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return dataSourceFactory.getPartitionConstraint();
+    }
+
+    /**
+     * Runs on each node controller (after serialization-deserialization)
+     */
+    @Override
+    public IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+        IDataFlowController controller = DataflowControllerProvider.getDataflowController(recordType, ctx, partition,
+                dataSourceFactory, dataParserFactory, configuration, indexingOp);
+        return new GenericAdapter(controller);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+        this.recordType = outputType;
+        this.configuration = configuration;
+        dataSourceFactory = DatasourceFactoryProvider.getExternalDataSourceFactory(configuration);
+        dataParserFactory = ParserFactoryProvider.getDataParserFactory(configuration);
+        prepare();
+        ExternalDataCompatibilityUtils.validateCompatibility(dataSourceFactory, dataParserFactory);
+    }
+
+    private void prepare() throws Exception {
+        if (dataSourceFactory.isIndexible() && (files != null)) {
+            ((IIndexibleExternalDataSource) dataSourceFactory).setSnapshot(files, indexingOp);
+        }
+        dataSourceFactory.configure(configuration);
+        dataParserFactory.setRecordType(recordType);
+        dataParserFactory.configure(configuration);
+    }
+
+    @Override
+    public ARecordType getAdapterOutputType() {
+        return recordType;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSAdapterFactory.java
deleted file mode 100644
index c4a96f4..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ /dev/null
@@ -1,343 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.dataset.adapter.HDFSAdapter;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.dataflow.HDFSObjectTupleParserFactory;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.ICCContext;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.hdfs.dataflow.ConfFactory;
-import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
-import org.apache.hyracks.hdfs.scheduler.Scheduler;
-
-/**
- * A factory class for creating an instance of HDFSAdapter
- */
-public class HDFSAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final String HDFS_ADAPTER_NAME = "hdfs";
-    public static final String CLUSTER_LOCATIONS = "cluster-locations";
-    public static transient String SCHEDULER = "hdfs-scheduler";
-
-    public static final String KEY_HDFS_URL = "hdfs";
-    public static final String KEY_PATH = "path";
-    public static final String KEY_INPUT_FORMAT = "input-format";
-    public static final String INPUT_FORMAT_TEXT = "text-input-format";
-    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
-    // New
-    public static final String KEY_PARSER = "parser";
-    public static final String PARSER_HIVE = "hive-parser";
-    public static final String INPUT_FORMAT_RC = "rc-input-format";
-    public static final String FORMAT_BINARY = "binary";
-
-    public static final String KEY_LOCAL_SOCKET_PATH = "local-socket-path";
-
-    // Hadoop property names constants
-    public static final String CLASS_NAME_TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat";
-    public static final String CLASS_NAME_SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat";
-    public static final String CLASS_NAME_RC_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.RCFileInputFormat";
-    public static final String CLASS_NAME_HDFS_FILESYSTEM = "org.apache.hadoop.hdfs.DistributedFileSystem";
-    public static final String KEY_HADOOP_FILESYSTEM_URI = "fs.defaultFS";
-    public static final String KEY_HADOOP_FILESYSTEM_CLASS = "fs.hdfs.impl";
-    public static final String KEY_HADOOP_INPUT_DIR = "mapred.input.dir";
-    public static final String KEY_HADOOP_INPUT_FORMAT = "mapred.input.format.class";
-    public static final String KEY_HADOOP_SHORT_CIRCUIT = "dfs.client.read.shortcircuit";
-    public static final String KEY_HADOOP_SOCKET_PATH = "dfs.domain.socket.path";
-
-    private transient AlgebricksPartitionConstraint clusterLocations;
-    private String[] readSchedule;
-    private boolean executed[];
-    private InputSplitsFactory inputSplitsFactory;
-    private ConfFactory confFactory;
-    private IAType atype;
-    private boolean configured = false;
-    public static Scheduler hdfsScheduler;
-    private static boolean initialized = false;
-    protected List<ExternalFile> files;
-
-    private static Scheduler initializeHDFSScheduler() {
-        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
-        Scheduler scheduler = null;
-        try {
-            scheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
-                    ccContext.getClusterControllerInfo().getClientNetPort());
-        } catch (HyracksException e) {
-            throw new IllegalStateException("Cannot obtain hdfs scheduler");
-        }
-        return scheduler;
-    }
-
-    protected static final Map<String, String> formatClassNames = initInputFormatMap();
-
-    protected static Map<String, String> initInputFormatMap() {
-        Map<String, String> formatClassNames = new HashMap<String, String>();
-        formatClassNames.put(INPUT_FORMAT_TEXT, CLASS_NAME_TEXT_INPUT_FORMAT);
-        formatClassNames.put(INPUT_FORMAT_SEQUENCE, CLASS_NAME_SEQUENCE_INPUT_FORMAT);
-        formatClassNames.put(INPUT_FORMAT_RC, CLASS_NAME_RC_INPUT_FORMAT);
-        return formatClassNames;
-    }
-
-    public JobConf getJobConf() throws HyracksDataException {
-        return confFactory.getConf();
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        JobConf conf = confFactory.getConf();
-        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
-        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName,
-                parserFactory, ctx, configuration, files);
-        return hdfsAdapter;
-    }
-
-    @Override
-    public String getName() {
-        return HDFS_ADAPTER_NAME;
-    }
-
-    public static JobConf configureJobConf(Map<String, String> configuration) throws Exception {
-        JobConf conf = new JobConf();
-        String formatClassName = formatClassNames.get(configuration.get(KEY_INPUT_FORMAT).trim());
-        String localShortCircuitSocketPath = configuration.get(KEY_LOCAL_SOCKET_PATH);
-        if (formatClassName == null) {
-            formatClassName = configuration.get(KEY_INPUT_FORMAT).trim();
-        }
-        conf.set(KEY_HADOOP_FILESYSTEM_URI, configuration.get(KEY_HDFS_URL).trim());
-        conf.set(KEY_HADOOP_FILESYSTEM_CLASS, CLASS_NAME_HDFS_FILESYSTEM);
-        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-        conf.set(KEY_HADOOP_INPUT_DIR, configuration.get(KEY_PATH).trim());
-        conf.set(KEY_HADOOP_INPUT_FORMAT, formatClassName);
-
-        // Enable local short circuit reads if user supplied the parameters
-        if (localShortCircuitSocketPath != null) {
-            conf.set(KEY_HADOOP_SHORT_CIRCUIT, "true");
-            conf.set(KEY_HADOOP_SOCKET_PATH, localShortCircuitSocketPath.trim());
-        }
-        return conf;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (!configured) {
-            throw new IllegalStateException("Adapter factory has not been configured yet");
-        }
-        return clusterLocations;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        if (!initialized) {
-            hdfsScheduler = initializeHDFSScheduler();
-            initialized = true;
-        }
-        this.configuration = configuration;
-        JobConf conf = configureJobConf(configuration);
-        confFactory = new ConfFactory(conf);
-
-        clusterLocations = getClusterLocations();
-        int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
-        // if files list was set, we restrict the splits to the list since this dataset is indexed
-        InputSplit[] inputSplits;
-        if (files == null) {
-            inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-        } else {
-            inputSplits = getSplits(conf);
-        }
-        inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
-        readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
-        executed = new boolean[readSchedule.length];
-        Arrays.fill(executed, false);
-        configured = true;
-
-        atype = outputType;
-        configureFormat(atype);
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    public static AlgebricksPartitionConstraint getClusterLocations() {
-        ArrayList<String> locs = new ArrayList<String>();
-        Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
-        for (String i : stores.keySet()) {
-            String[] nodeStores = stores.get(i);
-            for (int j = 0; j < nodeStores.length; j++) {
-                //two readers per partition
-                locs.add(i);
-                locs.add(i);
-            }
-        }
-        String[] cluster = new String[locs.size()];
-        cluster = locs.toArray(cluster);
-        return new AlgebricksAbsolutePartitionConstraint(cluster);
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return (ARecordType) atype;
-    }
-
-    @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.UNKNOWN;
-    }
-
-    /*
-     * This method is overridden to do the following:
-     * if data is text data (adm or delimited text), it will use a text tuple parser,
-     * otherwise it will use hdfs record object parser
-     */
-    @Override
-    protected void configureFormat(IAType sourceDatatype) throws Exception {
-        String specifiedFormat = configuration.get(AsterixTupleParserFactory.KEY_FORMAT);
-        if (specifiedFormat == null) {
-            throw new IllegalArgumentException(" Unspecified data format");
-        }
-
-        if (AsterixTupleParserFactory.FORMAT_BINARY.equalsIgnoreCase(specifiedFormat)) {
-            parserFactory = new HDFSObjectTupleParserFactory((ARecordType) atype, this, configuration);
-        } else {
-            InputDataFormat inputFormat = InputDataFormat.UNKNOWN;
-            if (AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
-                inputFormat = InputDataFormat.DELIMITED;
-            } else if (AsterixTupleParserFactory.FORMAT_ADM.equalsIgnoreCase(specifiedFormat)) {
-                inputFormat = InputDataFormat.ADM;
-            }
-            parserFactory = new AsterixTupleParserFactory(configuration, (ARecordType) sourceDatatype, inputFormat);
-        }
-
-    }
-
-    /**
-     * Instead of creating the split using the input format, we do it manually
-     * This function returns fileSplits (1 per hdfs file block) irrespective of the number of partitions
-     * and the produced splits only cover intersection between current files in hdfs and files stored internally
-     * in AsterixDB
-     * 1. NoOp means appended file
-     * 2. AddOp means new file
-     * 3. UpdateOp means the delta of a file
-     *
-     * @return
-     * @throws IOException
-     */
-    protected InputSplit[] getSplits(JobConf conf) throws IOException {
-        ArrayList<FileSplit> fileSplits = new ArrayList<FileSplit>();
-        ArrayList<ExternalFile> orderedExternalFiles = new ArrayList<ExternalFile>();
-        // Create file system object
-        try (FileSystem fs = FileSystem.get(conf)) {
-            // Create files splits
-            for (ExternalFile file : files) {
-                Path filePath = new Path(file.getFileName());
-                FileStatus fileStatus;
-                try {
-                    fileStatus = fs.getFileStatus(filePath);
-                } catch (FileNotFoundException e) {
-                    // file was deleted at some point, skip to next file
-                    continue;
-                }
-                if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP
-                        && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
-                    // Get its information from HDFS name node
-                    BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, file.getSize());
-                    // Create a split per block
-                    for (BlockLocation block : fileBlocks) {
-                        if (block.getOffset() < file.getSize()) {
-                            fileSplits.add(new FileSplit(filePath,
-                                    block.getOffset(), (block.getLength() + block.getOffset()) < file.getSize()
-                                            ? block.getLength() : (file.getSize() - block.getOffset()),
-                                    block.getHosts()));
-                            orderedExternalFiles.add(file);
-                        }
-                    }
-                } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP
-                        && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
-                    long oldSize = 0L;
-                    long newSize = file.getSize();
-                    for (int i = 0; i < files.size(); i++) {
-                        if (files.get(i).getFileName() == file.getFileName()
-                                && files.get(i).getSize() != file.getSize()) {
-                            newSize = files.get(i).getSize();
-                            oldSize = file.getSize();
-                            break;
-                        }
-                    }
-
-                    // Get its information from HDFS name node
-                    BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, newSize);
-                    // Create a split per block
-                    for (BlockLocation block : fileBlocks) {
-                        if (block.getOffset() + block.getLength() > oldSize) {
-                            if (block.getOffset() < newSize) {
-                                // Block interact with delta -> Create a split
-                                long startCut = (block.getOffset() > oldSize) ? 0L : oldSize - block.getOffset();
-                                long endCut = (block.getOffset() + block.getLength() < newSize) ? 0L
-                                        : block.getOffset() + block.getLength() - newSize;
-                                long splitLength = block.getLength() - startCut - endCut;
-                                fileSplits.add(new FileSplit(filePath, block.getOffset() + startCut, splitLength,
-                                        block.getHosts()));
-                                orderedExternalFiles.add(file);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-        files = orderedExternalFiles;
-        return fileSplits.toArray(new FileSplit[fileSplits.size()]);
-    }
-
-    // Used to tell the factory to restrict the splits to the intersection between this list and the actual files on hdfs side
-    public void setFiles(List<ExternalFile> files) {
-        this.files = files;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
deleted file mode 100644
index 8bf6d93..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.dataset.adapter.HDFSIndexingAdapter;
-import org.apache.asterix.external.indexing.dataflow.HDFSIndexingParserFactory;
-import org.apache.asterix.external.indexing.dataflow.IndexingScheduler;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.DelimitedDataParser;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.api.context.ICCContext;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.hdfs.dataflow.ConfFactory;
-import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
-
-public class HDFSIndexingAdapterFactory extends HDFSAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private transient AlgebricksPartitionConstraint clusterLocations;
-    private String[] readSchedule;
-    private boolean executed[];
-    private InputSplitsFactory inputSplitsFactory;
-    private ConfFactory confFactory;
-    private IAType atype;
-    private boolean configured = false;
-    public static IndexingScheduler hdfsScheduler;
-    private static boolean initialized = false;
-    private Map<String, String> configuration;
-
-    public static final String HDFS_INDEXING_ADAPTER = "hdfs-indexing-adapter";
-
-    private static IndexingScheduler initializeHDFSScheduler() {
-        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
-        IndexingScheduler scheduler = null;
-        try {
-            scheduler = new IndexingScheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
-                    ccContext.getClusterControllerInfo().getClientNetPort());
-        } catch (HyracksException e) {
-            throw new IllegalStateException("Cannot obtain hdfs scheduler");
-        }
-        return scheduler;
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public String getName() {
-        return HDFS_INDEXING_ADAPTER;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (!configured) {
-            throw new IllegalStateException("Adapter factory has not been configured yet");
-        }
-        return clusterLocations;
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        JobConf conf = confFactory.getConf();
-        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
-        ((HDFSIndexingParserFactory) parserFactory).setJobConf(conf);
-        ((HDFSIndexingParserFactory) parserFactory).setArguments(configuration);
-        HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits,
-                conf, clusterLocations, files, parserFactory, ctx, nodeName,
-                (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
-                (String) configuration.get(AsterixTupleParserFactory.KEY_FORMAT));
-        return hdfsIndexingAdapter;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        if (!initialized) {
-            hdfsScheduler = initializeHDFSScheduler();
-            initialized = true;
-        }
-        this.configuration = configuration;
-        JobConf conf = HDFSAdapterFactory.configureJobConf(configuration);
-        confFactory = new ConfFactory(conf);
-        clusterLocations = getClusterLocations();
-        InputSplit[] inputSplits = getSplits(conf);
-        inputSplitsFactory = new InputSplitsFactory(inputSplits);
-        readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
-        executed = new boolean[readSchedule.length];
-        Arrays.fill(executed, false);
-        configured = true;
-        atype = outputType;
-        // The function below is overwritten to create indexing adapter factory instead of regular adapter factory
-        configureFormat(atype);
-    }
-
-    @Override
-    protected void configureFormat(IAType sourceDatatype) throws Exception {
-
-        char delimiter = AsterixTupleParserFactory.getDelimiter(configuration);
-        char quote = AsterixTupleParserFactory.getQuote(configuration, delimiter);
-
-        parserFactory = new HDFSIndexingParserFactory((ARecordType) atype,
-                configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
-                configuration.get(AsterixTupleParserFactory.KEY_FORMAT), delimiter, quote,
-                configuration.get(HDFSAdapterFactory.KEY_PARSER));
-    }
-
-    /**
-     * A static function that creates and return delimited text data parser
-     *
-     * @param recordType
-     *            (the record type to be parsed)
-     * @param delimiter
-     *            (the delimiter value)
-     * @return
-     */
-    public static DelimitedDataParser getDelimitedDataParser(ARecordType recordType, char delimiter, char quote) {
-        int n = recordType.getFieldTypes().length;
-        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
-        for (int i = 0; i < n; i++) {
-            ATypeTag tag = null;
-            if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
-                if (!NonTaggedFormatUtil.isOptional(recordType.getFieldTypes()[i])) {
-                    throw new NotImplementedException("Non-optional UNION type is not supported.");
-                }
-                tag = ((AUnionType) recordType.getFieldTypes()[i]).getNullableType().getTypeTag();
-            } else {
-                tag = recordType.getFieldTypes()[i].getTypeTag();
-            }
-            if (tag == null) {
-                throw new NotImplementedException("Failed to get the type information for field " + i + ".");
-            }
-            IValueParserFactory vpf = valueParserFactoryMap.get(tag);
-            if (vpf == null) {
-                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-            }
-            fieldParserFactories[i] = vpf;
-        }
-        return new DelimitedDataParser(recordType, fieldParserFactories, delimiter, quote, false);
-    }
-
-    public static AlgebricksPartitionConstraint getClusterLocations() {
-        ArrayList<String> locs = new ArrayList<String>();
-        Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
-        for (String i : stores.keySet()) {
-            String[] nodeStores = stores.get(i);
-            for (int j = 0; j < nodeStores.length; j++) {
-                locs.add(i);
-            }
-        }
-        String[] cluster = new String[locs.size()];
-        cluster = locs.toArray(cluster);
-        return new AlgebricksAbsolutePartitionConstraint(cluster);
-    }
-
-    private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
-
-    private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
-        Map<ATypeTag, IValueParserFactory> m = new HashMap<ATypeTag, IValueParserFactory>();
-        m.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
-        m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        m.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
-        m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-        return m;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HiveAdapterFactory.java
deleted file mode 100644
index 553682e..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.dataset.adapter.HDFSAdapter;
-import org.apache.asterix.external.dataset.adapter.HiveAdapter;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * A factory class for creating an instance of HiveAdapter
- */
-public class HiveAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final String HIVE_DATABASE = "database";
-    public static final String HIVE_TABLE = "table";
-    public static final String HIVE_HOME = "hive-home";
-    public static final String HIVE_METASTORE_URI = "metastore-uri";
-    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
-    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
-
-    private HDFSAdapterFactory hdfsAdapterFactory;
-    private HDFSAdapter hdfsAdapter;
-    private boolean configured = false;
-    private IAType atype;
-
-    public HiveAdapterFactory() {
-        hdfsAdapterFactory = new HDFSAdapterFactory();
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        hdfsAdapter = (HDFSAdapter) hdfsAdapterFactory.createAdapter(ctx, partition);
-        HiveAdapter hiveAdapter = new HiveAdapter(atype, hdfsAdapter, parserFactory, ctx);
-        return hiveAdapter;
-    }
-
-    @Override
-    public String getName() {
-        return "hive";
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        if (!configured) {
-            populateConfiguration(configuration);
-            hdfsAdapterFactory.configure(configuration, outputType);
-            this.atype = outputType;
-        }
-    }
-
-    public static void populateConfiguration(Map<String, String> configuration) throws Exception {
-        /** configure hive */
-        String database = configuration.get(HIVE_DATABASE);
-        String tablePath = null;
-        if (database == null) {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
-        } else {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
-                    + configuration.get(HIVE_TABLE);
-        }
-        configuration.put(HDFSAdapterFactory.KEY_PATH, tablePath);
-        if (!configuration.get(AsterixTupleParserFactory.KEY_FORMAT)
-                .equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
-            throw new IllegalArgumentException(
-                    "format" + configuration.get(AsterixTupleParserFactory.KEY_FORMAT) + " is not supported");
-        }
-
-        if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)
-                || configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT)
-                        .equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))) {
-            throw new IllegalArgumentException(
-                    "file input format" + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
-        }
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return hdfsAdapterFactory.getPartitionConstraint();
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return (ARecordType) atype;
-    }
-
-    @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.UNKNOWN;
-    }
-
-    public void setFiles(List<ExternalFile> files) {
-        hdfsAdapterFactory.setFiles(files);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IControlledAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IControlledAdapterFactory.java
deleted file mode 100644
index 0de6fad..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IControlledAdapterFactory.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.asterix.external.dataset.adapter.IControlledAdapter;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-
-public interface IControlledAdapterFactory extends Serializable {
-    public IControlledAdapter createAdapter(IHyracksTaskContext ctx, ExternalFileIndexAccessor fileIndexAccessor,
-            RecordDescriptor inRecDesc);
-
-    public void configure(IAType atype, boolean propagateInput, int[] ridFields,
-            Map<String, String> adapterConfiguration, boolean retainNull);
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
new file mode 100644
index 0000000..866910b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.adapter.factory;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.asterix.external.api.ILookupReaderFactory;
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.dataset.adapter.LookupAdapter;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordIdReader;
+import org.apache.asterix.external.indexing.RecordIdReaderFactory;
+import org.apache.asterix.external.input.record.reader.LookupReaderFactoryProvider;
+import org.apache.asterix.external.provider.ParserFactoryProvider;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class LookupAdapterFactory<T> implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private IRecordDataParserFactory dataParserFactory;
+    private ILookupReaderFactory readerFactory;
+    private ARecordType recordType;
+    private int[] ridFields;
+    private Map<String, String> configuration;
+    private boolean retainInput;
+    private boolean retainNull;
+    private int[] propagatedFields;
+    private INullWriterFactory iNullWriterFactory;
+
+    public LookupAdapterFactory(ARecordType recordType, int[] ridFields, boolean retainInput, boolean retainNull,
+            INullWriterFactory iNullWriterFactory) {
+        this.recordType = recordType;
+        this.ridFields = ridFields;
+        this.retainInput = retainInput;
+        this.retainNull = retainNull;
+        this.iNullWriterFactory = iNullWriterFactory;
+    }
+
+    public LookupAdapter<T> createAdapter(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecDesc,
+            ExternalFileIndexAccessor snapshotAccessor, IFrameWriter writer) throws HyracksDataException {
+        try {
+            IRecordDataParser<T> dataParser = dataParserFactory.createRecordParser(ctx);
+            dataParser.configure(configuration, recordType);
+            ILookupRecordReader<? extends T> reader = readerFactory.createRecordReader(ctx, partition,
+                    snapshotAccessor);
+            reader.configure(configuration);
+            RecordIdReader ridReader = RecordIdReaderFactory.create(configuration, ridFields);
+            configurePropagatedFields(inRecDesc);
+            return new LookupAdapter<T>(dataParser, reader, inRecDesc, ridReader, retainInput, propagatedFields,
+                    retainNull, iNullWriterFactory, ctx, writer);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        readerFactory = LookupReaderFactoryProvider.getLookupReaderFactory(configuration);
+        dataParserFactory = (IRecordDataParserFactory<T>) ParserFactoryProvider.getDataParserFactory(configuration);
+        dataParserFactory.setRecordType(recordType);
+        readerFactory.configure(configuration);
+        dataParserFactory.configure(configuration);
+    }
+
+    private void configurePropagatedFields(RecordDescriptor inRecDesc) {
+        int ptr = 0;
+        boolean skip = false;
+        propagatedFields = new int[inRecDesc.getFieldCount() - ridFields.length];
+        for (int i = 0; i < inRecDesc.getFieldCount(); i++) {
+            if (ptr < ridFields.length) {
+                skip = false;
+                for (int j = 0; j < ridFields.length; j++) {
+                    if (ridFields[j] == i) {
+                        ptr++;
+                        skip = true;
+                        break;
+                    }
+                }
+                if (!skip)
+                    propagatedFields[i - ptr] = i;
+            } else {
+                propagatedFields[i - ptr] = i;
+            }
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java
deleted file mode 100644
index fbde1b4..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.FeedPolicyAccessor;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.dataset.adapter.PullBasedAzureTwitterAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class PullBasedAzureTwitterAdapterFactory implements IFeedAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final String INGESTOR_LOCATIONS_KEY = "ingestor-locations";
-    private static final String PARTITIONS_KEY = "partitions";
-    private static final String TABLE_NAME_KEY = "table-name";
-    private static final String ACCOUNT_NAME_KEY = "account-name";
-    private static final String ACCOUNT_KEY_KEY = "account-key";
-
-    private ARecordType outputType;
-    private Map<String, String> configuration;
-    private String tableName;
-    private String azureAccountName;
-    private String azureAccountKey;
-    private String[] locations;
-    private String[] partitions;
-    private FeedPolicyAccessor ingestionPolicy;
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public String getName() {
-        return "azure_twitter";
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        String locationsStr = configuration.get(INGESTOR_LOCATIONS_KEY);
-        if (locationsStr == null) {
-            return null;
-        }
-        String[] locations = locationsStr.split(",");
-        return new AlgebricksAbsolutePartitionConstraint(locations);
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        return new PullBasedAzureTwitterAdapter(azureAccountName, azureAccountKey, tableName, partitions, configuration,
-                ctx, outputType);
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.configuration = configuration;
-        this.outputType = outputType;
-
-        tableName = configuration.get(TABLE_NAME_KEY);
-        if (tableName == null) {
-            throw new AsterixException("You must specify a valid table name");
-        }
-        azureAccountName = configuration.get(ACCOUNT_NAME_KEY);
-        azureAccountKey = configuration.get(ACCOUNT_KEY_KEY);
-        if (azureAccountName == null || azureAccountKey == null) {
-            throw new AsterixException("You must specify a valid Azure account name and key");
-        }
-
-        int nIngestLocations = 1;
-        String locationsStr = configuration.get(INGESTOR_LOCATIONS_KEY);
-        if (locationsStr != null) {
-            locations = locationsStr.split(",");
-            nIngestLocations = locations.length;
-        }
-
-        int nPartitions = 1;
-        String partitionsStr = configuration.get(PARTITIONS_KEY);
-        if (partitionsStr != null) {
-            partitions = partitionsStr.split(",");
-            nPartitions = partitions.length;
-        }
-
-        if (nIngestLocations != nPartitions) {
-            throw new AsterixException("Invalid adapter configuration: number of ingestion-locations ("
-                    + nIngestLocations + ") must be the same as the number of partitions (" + nPartitions + ")");
-        }
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
-    public FeedPolicyAccessor getIngestionPolicy() {
-        return ingestionPolicy;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
deleted file mode 100644
index 7d2dd73..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.dataset.adapter.PullBasedTwitterAdapter;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * Factory class for creating an instance of PullBasedTwitterAdapter.
- * This adapter provides the functionality of fetching tweets from Twitter service
- * via pull-based Twitter API.
- */
-public class PullBasedTwitterAdapterFactory implements IFeedAdapterFactory {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(PullBasedTwitterAdapterFactory.class.getName());
-
-    public static final String PULL_BASED_TWITTER_ADAPTER_NAME = "pull_twitter";
-
-    private static final String DEFAULT_INTERVAL = "10"; // 10 seconds
-    private static final int INTAKE_CARDINALITY = 1; // degree of parallelism at intake stage
-
-    private ARecordType outputType;
-
-    private Map<String, String> configuration;
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        return new PullBasedTwitterAdapter(configuration, outputType, ctx);
-    }
-
-    @Override
-    public String getName() {
-        return PULL_BASED_TWITTER_ADAPTER_NAME;
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.outputType = outputType;
-        this.configuration = configuration;
-        TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
-
-        if (configuration.get(SearchAPIConstants.QUERY) == null) {
-            throw new AsterixException(
-                    "parameter " + SearchAPIConstants.QUERY + " not specified as part of adaptor configuration");
-        }
-
-        String interval = configuration.get(SearchAPIConstants.INTERVAL);
-        if (interval != null) {
-            try {
-                Integer.parseInt(interval);
-            } catch (NumberFormatException nfe) {
-                throw new IllegalArgumentException(
-                        "parameter " + SearchAPIConstants.INTERVAL + " is defined incorrectly, expecting a number");
-            }
-        } else {
-            configuration.put(SearchAPIConstants.INTERVAL, DEFAULT_INTERVAL);
-            if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning(" Parameter " + SearchAPIConstants.INTERVAL + " not defined, using default ("
-                        + DEFAULT_INTERVAL + ")");
-            }
-        }
-
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(INTAKE_CARDINALITY);
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java
deleted file mode 100644
index 5bfdbcf..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.dataset.adapter.PushBasedTwitterAdapter;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.external.util.TwitterUtil.AuthenticationConstants;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class PushBasedTwitterAdapterFactory implements IFeedAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final String NAME = "push_twitter";
-
-    private ARecordType outputType;
-
-    private Map<String, String> configuration;
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public String getName() {
-        return NAME;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(1);
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        PushBasedTwitterAdapter twitterAdapter = new PushBasedTwitterAdapter(configuration, outputType, ctx);
-        return twitterAdapter;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.outputType = outputType;
-        this.configuration = configuration;
-        TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
-        boolean requiredParamsSpecified = validateConfiguration(configuration);
-        if (!requiredParamsSpecified) {
-            StringBuilder builder = new StringBuilder();
-            builder.append("One or more parameters are missing from adapter configuration\n");
-            builder.append(AuthenticationConstants.OAUTH_CONSUMER_KEY + "\n");
-            builder.append(AuthenticationConstants.OAUTH_CONSUMER_SECRET + "\n");
-            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN + "\n");
-            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET + "\n");
-            throw new Exception(builder.toString());
-        }
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
-    private boolean validateConfiguration(Map<String, String> configuration) {
-        String consumerKey = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_KEY);
-        String consumerSecret = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_SECRET);
-        String accessToken = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN);
-        String tokenSecret = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET);
-
-        if (consumerKey == null || consumerSecret == null || accessToken == null || tokenSecret == null) {
-            return false;
-        }
-        return true;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/RSSFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
deleted file mode 100644
index 4d893fe..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.FeedPolicyAccessor;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.dataset.adapter.RSSFeedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * Factory class for creating an instance of @see {RSSFeedAdapter}.
- * RSSFeedAdapter provides the functionality of fetching an RSS based feed.
- */
-public class RSSFeedAdapterFactory implements IFeedAdapterFactory {
-    private static final long serialVersionUID = 1L;
-    public static final String RSS_FEED_ADAPTER_NAME = "rss_feed";
-
-    public static final String KEY_RSS_URL = "url";
-    public static final String KEY_INTERVAL = "interval";
-
-    private Map<String, String> configuration;
-    private ARecordType outputType;
-    private List<String> feedURLs = new ArrayList<String>();
-    private FeedPolicyAccessor ingestionPolicy;
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter(configuration, outputType, ctx);
-        return rssFeedAdapter;
-    }
-
-    @Override
-    public String getName() {
-        return "rss_feed";
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.configuration = configuration;
-        this.outputType = outputType;
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
-        if (rssURLProperty == null) {
-            throw new IllegalArgumentException("no rss url provided");
-        }
-        initializeFeedURLs(rssURLProperty);
-        configurePartitionConstraints();
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(feedURLs.size());
-    }
-
-    private void initializeFeedURLs(String rssURLProperty) {
-        feedURLs.clear();
-        String[] feedURLProperty = rssURLProperty.split(",");
-        for (String feedURL : feedURLProperty) {
-            feedURLs.add(feedURL);
-        }
-    }
-
-    protected void configurePartitionConstraints() {
-
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
-    public FeedPolicyAccessor getIngestionPolicy() {
-        return ingestionPolicy;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/StreamBasedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
deleted file mode 100644
index c7e582f..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.adapter.factory;
-
-import java.util.Map;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.util.INodeResolver;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public abstract class StreamBasedAdapterFactory implements IAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-    protected static final Logger LOGGER = Logger.getLogger(StreamBasedAdapterFactory.class.getName());
-
-    protected static INodeResolver nodeResolver;
-
-    protected Map<String, String> configuration;
-    protected ITupleParserFactory parserFactory;
-
-    public abstract InputDataFormat getInputDataFormat();
-
-    protected void configureFormat(IAType sourceDatatype) throws Exception {
-        parserFactory = new AsterixTupleParserFactory(configuration, (ARecordType) sourceDatatype,
-                getInputDataFormat());
-
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java
similarity index 80%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java
index b8005cd..9539278 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java
@@ -16,12 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.adapter.factory;
+package org.apache.asterix.external.api;
 
 import java.io.Serializable;
 import java.util.Map;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -33,30 +33,12 @@
  */
 public interface IAdapterFactory extends Serializable {
 
-    public static final String KEY_TYPE_NAME = "type-name";
-
-    public enum SupportedOperation {
-        READ,
-        WRITE,
-        READ_WRITE
-    }
-
-    /**
-     * Returns the type of adapter indicating if the adapter can be used for
-     * reading from an external data source or writing to an external data
-     * source or can be used for both purposes.
-     * 
-     * @see SupportedOperation
-     * @return
-     */
-    public SupportedOperation getSupportedOperations();
-
     /**
      * Returns the display name corresponding to the Adapter type that is created by the factory.
      * 
      * @return the display name
      */
-    public String getName();
+    public String getAlias();
 
     /**
      * Gets a list of partition constraints. A partition constraint can be a
@@ -79,7 +61,7 @@
      * @return An instance of IDatasourceAdapter.
      * @throws Exception
      */
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception;
+    public IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception;
 
     /**
      * @param configuration
@@ -94,5 +76,4 @@
      * @return
      */
     public ARecordType getAdapterOutputType();
-
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java
new file mode 100644
index 0000000..f5f47ec
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.parse.ITupleForwarder;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDataFlowController {
+
+    /**
+     * Order of calls:
+     * 1. Constructor()
+     * 2. if record flow controller
+     * |-a. Set record reader
+     * |-b. Set record parser
+     * else
+     * |-a. Set stream parser
+     * 3. setTupleForwarder(forwarder)
+     * 4. configure(configuration,ctx)
+     * 5. start(writer)
+     */
+
+    public void start(IFrameWriter writer) throws HyracksDataException;
+
+    public boolean stop();
+
+    public boolean handleException(Throwable th);
+
+    public ITupleForwarder getTupleForwarder();
+
+    public void setTupleForwarder(ITupleForwarder forwarder);
+
+    public void configure(Map<String, String> configuration, IHyracksTaskContext ctx) throws IOException;
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
new file mode 100644
index 0000000..a4a5a43
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableOrderedList;
+import org.apache.asterix.om.base.AMutableRecord;
+import org.apache.asterix.om.base.AMutableUnorderedList;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public interface IDataParser {
+
+    /**
+     * @return The supported data sources
+     */
+    public DataSourceType getDataSourceType();
+
+    /**
+     * @param configuration
+     *            a set of configurations that comes from two sources.
+     *            1. The create adapter statement.
+     *            2. The query compiler.
+     * @param recordType
+     *            The expected record type
+     * @throws HyracksDataException
+     * @throws IOException
+     */
+    public void configure(Map<String, String> configuration, ARecordType recordType)
+            throws HyracksDataException, IOException;
+
+    /*
+     * The following two static methods are expensive. right now, they are used by RSSFeeds and Twitter feed
+     * TODO: Get rid of them
+     */
+    public static void writeRecord(AMutableRecord record, DataOutput dataOutput, IARecordBuilder recordBuilder)
+            throws IOException, AsterixException {
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        int numFields = record.getType().getFieldNames().length;
+        for (int pos = 0; pos < numFields; pos++) {
+            fieldValue.reset();
+            IAObject obj = record.getValueByPos(pos);
+            IDataParser.writeObject(obj, fieldValue.getDataOutput());
+            recordBuilder.addField(pos, fieldValue);
+        }
+        recordBuilder.write(dataOutput, true);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void writeObject(IAObject obj, DataOutput dataOutput) throws IOException, AsterixException {
+        switch (obj.getType().getTypeTag()) {
+            case RECORD: {
+                IARecordBuilder recordBuilder = new RecordBuilder();
+                recordBuilder.reset((ARecordType) obj.getType());
+                recordBuilder.init();
+                writeRecord((AMutableRecord) obj, dataOutput, recordBuilder);
+                break;
+            }
+
+            case ORDEREDLIST: {
+                OrderedListBuilder listBuilder = new OrderedListBuilder();
+                listBuilder.reset((AOrderedListType) ((AMutableOrderedList) obj).getType());
+                IACursor cursor = ((AMutableOrderedList) obj).getCursor();
+                ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
+                while (cursor.next()) {
+                    listItemValue.reset();
+                    IAObject item = cursor.get();
+                    writeObject(item, listItemValue.getDataOutput());
+                    listBuilder.addItem(listItemValue);
+                }
+                listBuilder.write(dataOutput, true);
+                break;
+            }
+
+            case UNORDEREDLIST: {
+                UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+                listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
+                IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
+                ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
+                while (cursor.next()) {
+                    listItemValue.reset();
+                    IAObject item = cursor.get();
+                    writeObject(item, listItemValue.getDataOutput());
+                    listBuilder.addItem(listItemValue);
+                }
+                listBuilder.write(dataOutput, true);
+                break;
+            }
+
+            default:
+                AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(obj.getType()).serialize(obj,
+                        dataOutput);
+                break;
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParserFactory.java
new file mode 100644
index 0000000..5c3845c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParserFactory.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.om.types.ARecordType;
+
+public interface IDataParserFactory extends Serializable {
+
+    /**
+     * @return The expected data source type {STREAM or RECORDS}
+     *         The data source type for a parser and a data source must match.
+     *         an instance of IDataParserFactory with RECORDS data source type must implement IRecordDataParserFactory
+     *         <T>
+     *         an instance of IDataParserFactory with STREAM data source type must implement IStreamDataParserFactory
+     * @throws AsterixException
+     */
+    public DataSourceType getDataSourceType() throws AsterixException;
+
+    /**
+     * Configure the data parser factory. The passed map contains key value pairs from the
+     * submitted AQL statement and any additional pairs added by the compiler
+     * @param configuration
+     */
+    public void configure(Map<String, String> configuration) throws Exception;
+
+    /**
+     * Set the record type expected to be produced by parsers created by this factory
+     * @param recordType
+     */
+    public void setRecordType(ARecordType recordType);
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
new file mode 100644
index 0000000..580ac99
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+
+public interface IExternalDataSourceFactory extends Serializable {
+
+    /**
+     * The data source type indicates whether the data source produces a continuous stream or
+     * a set of records
+     * @author amoudi
+     */
+    public enum DataSourceType {
+        STREAM,
+        RECORDS
+    }
+
+    /**
+     * @return The data source type {STREAM or RECORDS}
+     */
+    public DataSourceType getDataSourceType();
+
+    /**
+     * Specifies on which locations this data source is expected to run.
+     * @return
+     * @throws Exception
+     */
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
+
+    /**
+     * Configure the data parser factory. The passed map contains key value pairs from the
+     * submitted AQL statement and any additional pairs added by the compiler
+     * @param configuration
+     * @throws Exception
+     */
+    public void configure(Map<String, String> configuration) throws Exception;
+
+    /**
+     * Specify whether the external data source can be indexed
+     * @return
+     */
+    public boolean isIndexible();
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalFunction.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalFunction.java
similarity index 95%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalFunction.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalFunction.java
index 8e4cdbf..9c25c7f 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalFunction.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalFunction.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
+package org.apache.asterix.external.api;
 
 public interface IExternalFunction {
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalIndexer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalIndexer.java
new file mode 100644
index 0000000..0b4277e
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalIndexer.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+/**
+ * @author amoudi
+ *         This Interface represents the component responsible for adding record ids to tuples when indexing external data
+ */
+public interface IExternalIndexer extends Serializable {
+
+    /**
+     * This method is called by an indexible datasource when the external source reader have been updated.
+     * this gives a chance for the indexer to update its reader specific values (i,e. file name)
+     * @param reader
+     *        the new reader
+     * @throws Exception
+     */
+    public void reset(IRecordReader<?> reader) throws Exception;
+
+    /**
+     * This method is called by the dataflow controller with each tuple. the indexer is expected to append record ids to the tuple.
+     * @param tb
+     * @throws Exception
+     */
+    public void index(ArrayTupleBuilder tb) throws Exception;
+
+    /**
+     * This method returns the number of fields in the record id. It is used by tuple appender at the initialization step.
+     * @return
+     * @throws Exception
+     */
+    public int getNumberOfFields() throws Exception;
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalScalarFunction.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalScalarFunction.java
similarity index 95%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalScalarFunction.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalScalarFunction.java
index 8e82b83..ecdb833 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IExternalScalarFunction.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalScalarFunction.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
+package org.apache.asterix.external.api;
 
 public interface IExternalScalarFunction extends IExternalFunction {
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionFactory.java
similarity index 95%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionFactory.java
index cffdec6..5c5bcd0 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionFactory.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
+package org.apache.asterix.external.api;
 
 public interface IFunctionFactory {
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
similarity index 92%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
index 06d8f4a..ebd0757 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
@@ -16,10 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
+package org.apache.asterix.external.api;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.library.java.IJObject;
 import org.apache.asterix.external.library.java.JTypeTag;
 
 import java.io.IOException;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexibleExternalDataSource.java
similarity index 66%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexibleExternalDataSource.java
index 62052af..fe30b38 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexibleExternalDataSource.java
@@ -16,19 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feeds;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import java.util.List;
 
-public interface IPullBasedFeedAdapter extends IFeedAdapter {
+import org.apache.asterix.external.indexing.ExternalFile;
+
+public interface IIndexibleExternalDataSource extends IExternalDataSourceFactory {
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) throws Exception;
 
     /**
+     * Specify whether the external data source is configured for indexing
+     *
      * @return
      */
-    public FeedPolicyEnforcer getPolicyEnforcer();
-
-    /**
-     * @param feedPolicyEnforcer
-     */
-    public void setFeedPolicyEnforcer(FeedPolicyEnforcer feedPolicyEnforcer);
+    public boolean isIndexingOp();
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
similarity index 75%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
index 6fab922..37cc1cf 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.ATypeTag;
+import java.util.List;
 
-public interface IJType {
+import org.apache.asterix.external.indexing.ExternalFile;
 
-    public ATypeTag getTypeTag();
-
-    public IAObject getIAObject();
+public interface IIndexingAdapterFactory extends IAdapterFactory {
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp);
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingDatasource.java
similarity index 81%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingDatasource.java
index f602656..ed5e7b5 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingDatasource.java
@@ -16,11 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
+public interface IIndexingDatasource {
+    public IExternalIndexer getIndexer();
 
-public interface IFramePreprocessor {
-
-    public void preProcess(ByteBuffer frame) throws Exception;
+    public void setIndexer(IExternalIndexer indexer);
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProvider.java
similarity index 79%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProvider.java
index f602656..3dabb29 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProvider.java
@@ -16,11 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
+import org.apache.asterix.external.input.stream.AInputStream;
 
-public interface IFramePreprocessor {
-
-    public void preProcess(ByteBuffer frame) throws Exception;
+public interface IInputStreamProvider {
+    public AInputStream getInputStream() throws Exception;
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProviderFactory.java
similarity index 72%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProviderFactory.java
index 1dfbee9..3cc31dc 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IInputStreamProviderFactory.java
@@ -16,13 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+public interface IInputStreamProviderFactory extends IExternalDataSourceFactory {
 
-public interface IFramePostProcessor {
-
-    public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor);
+    public IInputStreamProvider createInputStreamProvider(IHyracksTaskContext ctx, int partition) throws Exception;
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJListAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJListAccessor.java
similarity index 91%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJListAccessor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IJListAccessor.java
index a467721..70665db 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJListAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJListAccessor.java
@@ -16,8 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
+import org.apache.asterix.external.library.java.JObjectPointableVisitor;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.container.IObjectPool;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObject.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObject.java
similarity index 96%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObject.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObject.java
index ffeacd4..a88f47d 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObject.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObject.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
 import java.io.DataOutput;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObjectAccessor.java
similarity index 95%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObjectAccessor.java
index e05c8b0..7b10af1 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJObjectAccessor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.IAType;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJRecordAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJRecordAccessor.java
similarity index 82%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJRecordAccessor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IJRecordAccessor.java
index ab22531..08c5dde 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJRecordAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJRecordAccessor.java
@@ -16,8 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
+import org.apache.asterix.external.library.java.JObjectPointableVisitor;
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.types.ARecordType;
@@ -27,7 +28,7 @@
 
 public interface IJRecordAccessor {
 
-    public JRecord access(ARecordVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool, ARecordType recordType,
-            JObjectPointableVisitor pointableVisitor) throws HyracksDataException;
+    public JRecord access(ARecordVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool,
+            ARecordType recordType, JObjectPointableVisitor pointableVisitor) throws HyracksDataException;
 
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJType.java
similarity index 94%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IJType.java
index 6fab922..9c0ebae 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJType.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IJType.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java
similarity index 65%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java
index 1dfbee9..c0baead 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java
@@ -16,13 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public interface IFramePostProcessor {
-
-    public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor);
-}
+public interface ILookupReaderFactory<T> extends IExternalDataSourceFactory {
+    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
+            ExternalFileIndexAccessor snapshotAccessor) throws Exception;
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.java
old mode 100755
new mode 100644
similarity index 61%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.java
index 06d8f4a..fd03a07
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IFunctionHelper.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.java
@@ -16,25 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.library.java.IJObject;
-import org.apache.asterix.external.library.java.JTypeTag;
+package org.apache.asterix.external.api;
 
 import java.io.IOException;
+import java.util.Map;
 
-public interface IFunctionHelper {
+import org.apache.asterix.external.indexing.RecordId;
 
-    public IJObject getArgument(int index);
+public interface ILookupRecordReader<T> {
 
-    public IJObject getResultObject();
+    public void configure(Map<String, String> configuration) throws Exception;
 
-    public void setResult(IJObject result) throws IOException, AsterixException;
+    public Class<?> getRecordClass() throws IOException;
 
-    public boolean isValidResult();
+    public IRawRecord<T> read(RecordId rid) throws IOException, Exception;
 
-    public IJObject getObject(JTypeTag jtypeTag);
+    public void open() throws IOException;
 
-    public void reset();
+    public void fail() throws IOException;
+
+    public void close() throws IOException;
+
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolver.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolver.java
similarity index 96%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolver.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolver.java
index 3a92b97..831cd69 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolver.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolver.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.util;
+package org.apache.asterix.external.api;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolverFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolverFactory.java
similarity index 96%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolverFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolverFactory.java
index b3c459b..ce49596 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/INodeResolverFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/INodeResolverFactory.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.util;
+package org.apache.asterix.external.api;
 
 /**
  * Factory for creating an instance of INodeResolver
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
new file mode 100644
index 0000000..92b500d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+/**
+ * This interface represents a raw record that is not parsed yet.
+ * @param <T>
+ */
+public interface IRawRecord<T> {
+    /**
+     * @return the bytes representing this record. This is intended to be used for passing raw records in frames and
+     *         performing lazy deserialization on them. If the record can't be serialized, this method returns null.
+     */
+    public byte[] getBytes();
+
+    /**
+     * @return the java object of the record.
+     */
+    public T get();
+
+    /**
+     * @return The class of the record objects.
+     */
+    public Class<?> getRecordClass();
+
+    /**
+     * Resets the object to prepare it for another write operation.
+     */
+    public void reset();
+
+    /**
+     * @return The size of the valid bytes of the object. If the object can't be serialized, this method returns -1
+     */
+    int size();
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
similarity index 69%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
index 62052af..cc24847 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feeds/IPullBasedFeedAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feeds;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import java.io.DataOutput;
 
-public interface IPullBasedFeedAdapter extends IFeedAdapter {
+public interface IRecordDataParser<T> extends IDataParser {
 
     /**
-     * @return
+     * @param record
+     * @param out
+     * @throws Exception
      */
-    public FeedPolicyEnforcer getPolicyEnforcer();
+    public void parse(IRawRecord<? extends T> record, DataOutput out) throws Exception;
 
     /**
-     * @param feedPolicyEnforcer
+     * @return the record class
      */
-    public void setFeedPolicyEnforcer(FeedPolicyEnforcer feedPolicyEnforcer);
+    public Class<? extends T> getRecordClass();
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParserFactory.java
similarity index 65%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParserFactory.java
index e05c8b0..993d947 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParserFactory.java
@@ -16,14 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.container.IObjectPool;
+import java.io.IOException;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public interface IJObjectAccessor {
-    IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> obj) throws HyracksDataException;
+public interface IRecordDataParserFactory<T> extends IDataParserFactory {
+    public IRecordDataParser<T> createRecordParser(IHyracksTaskContext ctx)
+            throws HyracksDataException, AsterixException, IOException;
 
+    public Class<? extends T> getRecordClass();
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordFlowController.java
similarity index 75%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordFlowController.java
index f602656..c3bdc56 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordFlowController.java
@@ -16,11 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
+public interface IRecordFlowController<T> extends IDataFlowController {
 
-public interface IFramePreprocessor {
+    public void setRecordParser(IRecordDataParser<T> dataParser);
 
-    public void preProcess(ByteBuffer frame) throws Exception;
+    public void setRecordReader(IRecordReader<T> recordReader) throws Exception;
+
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
new file mode 100644
index 0000000..019fe8f
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * This interface represents a record reader that reads data from external source as a set of records
+ * @param <T>
+ */
+public interface IRecordReader<T> extends Closeable {
+
+    /**
+     * Configure the reader with the set of key/value pairs passed by the compiler
+     * @param configuration
+     *        the set of key/value pairs
+     * @throws Exception
+     *         when the reader can't be configured (i,e. due to incorrect configuration, unreachable source, etc.)
+     */
+    public void configure(Map<String, String> configuration) throws Exception;
+
+    /**
+     * @return true if the reader has more records remaining, false, otherwise.
+     * @throws Exception
+     *         if an error takes place
+     */
+    public boolean hasNext() throws Exception;
+
+    /**
+     * @return the object representing the next record.
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public IRawRecord<T> next() throws IOException, InterruptedException;
+
+    /**
+     * @return the class of the java objects representing the records. used to check compatibility between readers and
+     *         parsers.
+     * @throws IOException
+     */
+    public Class<? extends T> getRecordClass() throws IOException;
+
+    /**
+     * used to stop reader from producing more records.
+     * @return true if the connection to the external source has been suspended, false otherwise.
+     */
+    public boolean stop();
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
similarity index 70%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
index 9358a52..adb2602 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.adapter.factory;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 
-public interface IFeedAdapterFactory extends IAdapterFactory {
+public interface IRecordReaderFactory<T> extends IExternalDataSourceFactory {
 
-    public boolean isRecordTrackingEnabled();
+    public IRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception;
 
-    public IIntakeProgressTracker createIntakeProgressTracker();
+    public Class<? extends T> getRecordClass();
 
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IResultCollector.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
similarity index 97%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/library/IResultCollector.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
index 3efdad6..9f14ec0 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/IResultCollector.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library;
+package org.apache.asterix.external.api;
 
 import java.io.DataOutput;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParser.java
new file mode 100644
index 0000000..31d6317
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParser.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.DataOutput;
+import java.io.InputStream;
+
+public interface IStreamDataParser extends IDataParser {
+    /**
+     * Sets the inputStream for the parser. called only for parsers that support InputStreams
+     */
+    public void setInputStream(InputStream in) throws Exception;
+
+    /**
+     * Parse data into output AsterixDataModel binary records.
+     * Used with parsers that support stream sources
+     *
+     * @param out
+     *            DataOutput instance that for writing the parser output.
+     */
+
+    public boolean parse(DataOutput out) throws Exception;
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParserFactory.java
similarity index 69%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParserFactory.java
index e05c8b0..828f71e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/IJObjectAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamDataParserFactory.java
@@ -16,14 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java;
+package org.apache.asterix.external.api;
 
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public interface IJObjectAccessor {
-    IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> obj) throws HyracksDataException;
+public interface IStreamDataParserFactory extends IDataParserFactory {
 
+    public IStreamDataParser createInputStreamParser(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException, AsterixException;
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamFlowController.java
similarity index 81%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamFlowController.java
index f602656..d368c48 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IStreamFlowController.java
@@ -16,11 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.api;
 
-import java.nio.ByteBuffer;
-
-public interface IFramePreprocessor {
-
-    public void preProcess(ByteBuffer frame) throws Exception;
+public interface IStreamFlowController extends IDataFlowController {
+    public void setStreamParser(IStreamDataParser dataParser);
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java
new file mode 100644
index 0000000..d06161e
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataflow;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.parse.ITupleForwarder;
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractDataFlowController implements IDataFlowController {
+
+    protected ITupleForwarder tupleForwarder;
+    protected IHyracksTaskContext ctx;
+    protected Map<String, String> configuration;
+
+    @Override
+    public ITupleForwarder getTupleForwarder() {
+        return tupleForwarder;
+    }
+
+    @Override
+    public void setTupleForwarder(ITupleForwarder tupleForwarder) {
+        this.tupleForwarder = tupleForwarder;
+    }
+
+    protected void initializeTupleForwarder(IFrameWriter writer) throws HyracksDataException {
+        tupleForwarder.initialize(ctx, writer);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, IHyracksTaskContext ctx) throws IOException {
+        this.configuration = configuration;
+        this.ctx = ctx;
+    }
+}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/CounterTimerTupleForwarder.java
similarity index 88%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/CounterTimerTupleForwarder.java
index c06caef..116ec09 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/CounterTimerTupleForwarder.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.operators.file;
+package org.apache.asterix.external.dataflow;
 
 import java.util.Map;
 import java.util.Timer;
@@ -24,7 +24,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
+import org.apache.asterix.common.parse.ITupleForwarder;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
@@ -34,13 +34,13 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 
-public class CounterTimerTupleForwardPolicy implements ITupleForwardPolicy {
+public class CounterTimerTupleForwarder implements ITupleForwarder {
 
     public static final String BATCH_SIZE = "batch-size";
     public static final String BATCH_INTERVAL = "batch-interval";
 
-    private static final Logger LOGGER = Logger.getLogger(CounterTimerTupleForwardPolicy.class.getName());
-   
+    private static final Logger LOGGER = Logger.getLogger(CounterTimerTupleForwarder.class.getName());
+
     private FrameTupleAppender appender;
     private IFrame frame;
     private IFrameWriter writer;
@@ -52,21 +52,23 @@
     private Object lock = new Object();
     private boolean activeTimer = false;
 
+    @Override
     public void configure(Map<String, String> configuration) {
-        String propValue = (String) configuration.get(BATCH_SIZE);
+        String propValue = configuration.get(BATCH_SIZE);
         if (propValue != null) {
             batchSize = Integer.parseInt(propValue);
         } else {
             batchSize = -1;
         }
 
-        propValue = (String) configuration.get(BATCH_INTERVAL);
+        propValue = configuration.get(BATCH_INTERVAL);
         if (propValue != null) {
             batchInterval = Long.parseLong(propValue);
             activeTimer = true;
         }
     }
 
+    @Override
     public void initialize(IHyracksCommonContext ctx, IFrameWriter writer) throws HyracksDataException {
         this.appender = new FrameTupleAppender();
         this.frame = new VSizeFrame(ctx);
@@ -79,6 +81,7 @@
         }
     }
 
+    @Override
     public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
         if (activeTimer) {
             synchronized (lock) {
@@ -91,7 +94,8 @@
     }
 
     private void addTupleToFrame(ArrayTupleBuilder tb) throws HyracksDataException {
-        if (tuplesInFrame == batchSize || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+        if (tuplesInFrame == batchSize
+                || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("flushing frame containg (" + tuplesInFrame + ") tuples");
             }
@@ -104,6 +108,7 @@
         }
     }
 
+    @Override
     public void close() throws HyracksDataException {
         if (appender.getTupleCount() > 0) {
             if (activeTimer) {
@@ -149,10 +154,4 @@
         }
 
     }
-
-    @Override
-    public TupleForwardPolicyType getType() {
-        return TupleForwardPolicyType.COUNTER_TIMER_EXPIRED;
-    }
-
 }
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FrameFullTupleForwarder.java
similarity index 73%
copy from asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FrameFullTupleForwarder.java
index c5af720..36d41b4 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FrameFullTupleForwarder.java
@@ -16,11 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.operators.file;
+package org.apache.asterix.external.dataflow;
 
 import java.util.Map;
 
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
+import org.apache.asterix.common.parse.ITupleForwarder;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
@@ -30,24 +30,18 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 
-public class RateControlledTupleForwardPolicy implements ITupleForwardPolicy {
+public class FrameFullTupleForwarder implements ITupleForwarder {
 
     private FrameTupleAppender appender;
     private IFrame frame;
     private IFrameWriter writer;
-    private long interTupleInterval;
-    private boolean delayConfigured;
 
-    public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
-
+    @Override
     public void configure(Map<String, String> configuration) {
-        String propValue = configuration.get(INTER_TUPLE_INTERVAL);
-        if (propValue != null) {
-            interTupleInterval = Long.parseLong(propValue);
-        }
-        delayConfigured = interTupleInterval != 0;
+        // no-op
     }
 
+    @Override
     public void initialize(IHyracksCommonContext ctx, IFrameWriter writer) throws HyracksDataException {
         this.appender = new FrameTupleAppender();
         this.frame = new VSizeFrame(ctx);
@@ -55,14 +49,8 @@
         appender.reset(frame, true);
     }
 
+    @Override
     public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
-        if (delayConfigured) {
-            try {
-                Thread.sleep(interTupleInterval);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        }
         boolean success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
         if (!success) {
             FrameUtils.flushFrame(frame.getBuffer(), writer);
@@ -74,15 +62,11 @@
         }
     }
 
+    @Override
     public void close() throws HyracksDataException {
         if (appender.getTupleCount() > 0) {
             FrameUtils.flushFrame(frame.getBuffer(), writer);
         }
 
     }
-
-    @Override
-    public TupleForwardPolicyType getType() {
-        return TupleForwardPolicyType.RATE_CONTROLLED;
-    }
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/IndexingDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/IndexingDataFlowController.java
new file mode 100644
index 0000000..68c6f9b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/IndexingDataFlowController.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataflow;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class IndexingDataFlowController<T> extends RecordDataFlowController<T> {
+    IExternalIndexer indexer;
+
+    @Override
+    protected void appendOtherTupleFields(ArrayTupleBuilder tb) throws Exception {
+        indexer.index(tb);
+    }
+
+    @Override
+    public void setRecordReader(IRecordReader<T> recordReader) throws Exception {
+        super.setRecordReader(recordReader);
+        indexer = ((IIndexingDatasource) recordReader).getIndexer();
+        numOfTupleFields += indexer.getNumberOfFields();
+    }
+}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RateControlledTupleForwarder.java
similarity index 90%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RateControlledTupleForwarder.java
index c5af720..99cc3d1 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RateControlledTupleForwarder.java
@@ -16,11 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.operators.file;
+package org.apache.asterix.external.dataflow;
 
 import java.util.Map;
 
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
+import org.apache.asterix.common.parse.ITupleForwarder;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
@@ -30,7 +30,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 
-public class RateControlledTupleForwardPolicy implements ITupleForwardPolicy {
+public class RateControlledTupleForwarder implements ITupleForwarder {
 
     private FrameTupleAppender appender;
     private IFrame frame;
@@ -40,6 +40,7 @@
 
     public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
 
+    @Override
     public void configure(Map<String, String> configuration) {
         String propValue = configuration.get(INTER_TUPLE_INTERVAL);
         if (propValue != null) {
@@ -48,6 +49,7 @@
         delayConfigured = interTupleInterval != 0;
     }
 
+    @Override
     public void initialize(IHyracksCommonContext ctx, IFrameWriter writer) throws HyracksDataException {
         this.appender = new FrameTupleAppender();
         this.frame = new VSizeFrame(ctx);
@@ -55,6 +57,7 @@
         appender.reset(frame, true);
     }
 
+    @Override
     public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
         if (delayConfigured) {
             try {
@@ -74,15 +77,11 @@
         }
     }
 
+    @Override
     public void close() throws HyracksDataException {
         if (appender.getTupleCount() > 0) {
             FrameUtils.flushFrame(frame.getBuffer(), writer);
         }
 
     }
-
-    @Override
-    public TupleForwardPolicyType getType() {
-        return TupleForwardPolicyType.RATE_CONTROLLED;
-    }
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
new file mode 100644
index 0000000..ad8e791
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataflow;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordFlowController;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class RecordDataFlowController<T> extends AbstractDataFlowController implements IRecordFlowController<T> {
+
+    protected IRecordDataParser<T> dataParser;
+    protected IRecordReader<? extends T> recordReader;
+    protected int numOfTupleFields = 1;
+
+    @Override
+    public void start(IFrameWriter writer) throws HyracksDataException {
+        try {
+            ArrayTupleBuilder tb = new ArrayTupleBuilder(numOfTupleFields);
+            initializeTupleForwarder(writer);
+            while (recordReader.hasNext()) {
+                IRawRecord<? extends T> record = recordReader.next();
+                tb.reset();
+                dataParser.parse(record, tb.getDataOutput());
+                tb.addFieldEndOffset();
+                appendOtherTupleFields(tb);
+                tupleForwarder.addTuple(tb);
+            }
+            tupleForwarder.close();
+            recordReader.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    protected void appendOtherTupleFields(ArrayTupleBuilder tb) throws Exception {
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    @Override
+    public boolean handleException(Throwable th) {
+        return false;
+    }
+
+    @Override
+    public void setRecordParser(IRecordDataParser<T> dataParser) {
+        this.dataParser = dataParser;
+    }
+
+    @Override
+    public void setRecordReader(IRecordReader<T> recordReader) throws Exception {
+        this.recordReader = recordReader;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java
new file mode 100644
index 0000000..3016470
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataflow;
+
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.api.IStreamFlowController;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class StreamDataFlowController extends AbstractDataFlowController implements IStreamFlowController {
+    private IStreamDataParser dataParser;
+    private static final int NUMBER_OF_TUPLE_FIELDS = 1;
+
+    @Override
+    public void start(IFrameWriter writer) throws HyracksDataException {
+        try {
+            ArrayTupleBuilder tb = new ArrayTupleBuilder(NUMBER_OF_TUPLE_FIELDS);
+            initializeTupleForwarder(writer);
+            while (true) {
+                tb.reset();
+                if (!dataParser.parse(tb.getDataOutput())) {
+                    break;
+                }
+                tb.addFieldEndOffset();
+                tupleForwarder.addTuple(tb);
+            }
+            tupleForwarder.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    @Override
+    public boolean handleException(Throwable th) {
+        return false;
+    }
+
+    @Override
+    public void setStreamParser(IStreamDataParser dataParser) {
+        this.dataParser = dataParser;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetEntity.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetEntity.java
deleted file mode 100644
index ede820d..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetEntity.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import com.microsoft.windowsazure.services.table.client.TableServiceEntity;
-
-public class AzureTweetEntity extends TableServiceEntity {
-
-    private String postingType;
-    private String json;
-
-    public AzureTweetEntity() {
-    }
-
-    public AzureTweetEntity(String userID, String postingID) {
-        this.partitionKey = userID;
-        this.rowKey = postingID;
-    }
-
-    public String getPostingType() {
-        return postingType;
-    }
-
-    public void setPostingType(String postingType) {
-        this.postingType = postingType;
-    }
-
-    public void setJSON(String json) {
-        this.json = json;
-    }
-
-    public String getJSON() {
-        return json;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetMetadataEntity.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetMetadataEntity.java
deleted file mode 100644
index ddda897..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AzureTweetMetadataEntity.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import com.microsoft.windowsazure.services.table.client.TableServiceEntity;
-
-public class AzureTweetMetadataEntity extends TableServiceEntity {
-    private String creationTimestamp;
-    private String postingType;
-    private String productId;
-    private String ethnicity;
-    private String gender;
-    private String sentiment;
-    private String location;
-
-    public AzureTweetMetadataEntity() {
-    }
-
-    public AzureTweetMetadataEntity(String partitionKey, String rowKey) {
-        this.partitionKey = partitionKey;
-        this.rowKey = rowKey;
-    }
-
-    public String getCreationTimestamp() {
-        return creationTimestamp;
-    }
-
-    public void setCreationTimestamp(String creationTimestamp) {
-        this.creationTimestamp = creationTimestamp;
-    }
-
-    public String getPostingType() {
-        return postingType;
-    }
-
-    public void setPostingType(String postingType) {
-        this.postingType = postingType;
-    }
-
-    public String getProductId() {
-        return productId;
-    }
-
-    public void setProductId(String productId) {
-        this.productId = productId;
-    }
-
-    public String getEthnicity() {
-        return ethnicity;
-    }
-
-    public void setEthnicity(String ethnicity) {
-        this.ethnicity = ethnicity;
-    }
-
-    public String getGender() {
-        return gender;
-    }
-
-    public void setGender(String gender) {
-        this.gender = gender;
-    }
-
-    public String getSentiment() {
-        return sentiment;
-    }
-
-    public void setSentiment(String sentiment) {
-        this.sentiment = sentiment;
-    }
-
-    public String getLocation() {
-        return location;
-    }
-
-    public void setLocation(String location) {
-        this.location = location;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java
deleted file mode 100644
index a197368..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.external.dataset.adapter.IFeedClient.InflowState;
-import org.apache.asterix.external.feeds.FeedPolicyEnforcer;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-
-/**
- * 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 ClientBasedFeedAdapter implements IFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(ClientBasedFeedAdapter.class.getName());
-    private static final int timeout = 5; // seconds
-
-    protected ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
-    protected IFeedClient pullBasedFeedClient;
-    protected ARecordType adapterOutputType;
-    protected boolean continueIngestion = true;
-    protected Map<String, String> configuration;
-
-    private FrameTupleAppender appender;
-    private IFrame frame;
-    private long tupleCount = 0;
-    private final IHyracksTaskContext ctx;
-    private int frameTupleCount = 0;
-
-    protected FeedPolicyEnforcer policyEnforcer;
-
-    public FeedPolicyEnforcer getPolicyEnforcer() {
-        return policyEnforcer;
-    }
-
-    public void setFeedPolicyEnforcer(FeedPolicyEnforcer policyEnforcer) {
-        this.policyEnforcer = policyEnforcer;
-    }
-
-    public abstract IFeedClient getFeedClient(int partition) throws Exception;
-
-    public abstract ITupleForwardPolicy getTupleParserPolicy();
-
-    public ClientBasedFeedAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) {
-        this.ctx = ctx;
-        this.configuration = configuration;
-    }
-
-    public long getIngestedRecordsCount() {
-        return tupleCount;
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        appender = new FrameTupleAppender();
-        frame = new VSizeFrame(ctx);
-        appender.reset(frame, true);
-        ITupleForwardPolicy policy = getTupleParserPolicy();
-        policy.configure(configuration);
-        pullBasedFeedClient = getFeedClient(partition);
-        InflowState inflowState = null;
-        policy.initialize(ctx, writer);
-        while (continueIngestion) {
-            tupleBuilder.reset();
-            try {
-                inflowState = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput(), timeout);
-                switch (inflowState) {
-                    case DATA_AVAILABLE:
-                        tupleBuilder.addFieldEndOffset();
-                        policy.addTuple(tupleBuilder);
-                        frameTupleCount++;
-                        break;
-                    case NO_MORE_DATA:
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("Reached end of feed");
-                        }
-                        policy.close();
-                        tupleCount += frameTupleCount;
-                        frameTupleCount = 0;
-                        continueIngestion = false;
-                        break;
-                    case DATA_NOT_AVAILABLE:
-                        if (LOGGER.isLoggable(Level.WARNING)) {
-                            LOGGER.warning("Timed out on obtaining data from pull based adapter. Trying again!");
-                        }
-                        break;
-                }
-
-            } catch (Exception failureException) {
-                try {
-                    failureException.printStackTrace();
-                    boolean continueIngestion = policyEnforcer.continueIngestionPostSoftwareFailure(failureException);
-                    if (continueIngestion) {
-                        tupleBuilder.reset();
-                        continue;
-                    } else {
-                        throw failureException;
-                    }
-                } catch (Exception recoveryException) {
-                    throw new Exception(recoveryException);
-                }
-            }
-        }
-    }
-
-    /**
-     * Discontinue the ingestion of data and end the feed.
-     * 
-     * @throws Exception
-     */
-    @Override
-    public void stop() throws Exception {
-        continueIngestion = false;
-    }
-
-    public Map<String, String> getConfiguration() {
-        return configuration;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return false;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedClient.java
deleted file mode 100644
index e321b67..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedClient.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableOrderedList;
-import org.apache.asterix.om.base.AMutablePoint;
-import org.apache.asterix.om.base.AMutableRecord;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.AMutableUnorderedList;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.base.IACursor;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.AUnorderedListType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-
-public abstract class FeedClient implements IFeedClient {
-
-    protected static final Logger LOGGER = Logger.getLogger(FeedClient.class.getName());
-
-    protected ARecordSerializerDeserializer recordSerDe;
-    protected AMutableRecord mutableRecord;
-    protected boolean messageReceived;
-    protected boolean continueIngestion = true;
-    protected IARecordBuilder recordBuilder = new RecordBuilder();
-
-    protected AMutableString aString = new AMutableString("");
-    protected AMutableInt32 aInt32 = new AMutableInt32(0);
-    protected AMutablePoint aPoint = new AMutablePoint(0, 0);
-    protected AMutableDateTime aDateTime = new AMutableDateTime(0);
-
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ASTRING);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINT32);
-
-    public abstract InflowState retrieveNextRecord() throws Exception;
-
-    @Override
-    public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException {
-        try {
-            InflowState state = null;
-            int waitCount = 0;
-            boolean continueWait = true;
-            while ((state == null || state.equals(InflowState.DATA_NOT_AVAILABLE)) && continueWait) {
-                state = retrieveNextRecord();
-                switch (state) {
-                    case DATA_AVAILABLE:
-                        recordBuilder.reset(mutableRecord.getType());
-                        recordBuilder.init();
-                        writeRecord(mutableRecord, dataOutput, recordBuilder);
-                        break;
-                    case DATA_NOT_AVAILABLE:
-                        if (waitCount > timeout) {
-                            continueWait = false;
-                        } else {
-                            if (LOGGER.isLoggable(Level.WARNING)) {
-                                LOGGER.warning("Waiting to obtain data from pull based adaptor");
-                            }
-                            Thread.sleep(1000);
-                            waitCount++;
-                        }
-                        break;
-                    case NO_MORE_DATA:
-                        break;
-                }
-            }
-            return state;
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-
-    }
-
-    private void writeRecord(AMutableRecord record, DataOutput dataOutput, IARecordBuilder recordBuilder)
-            throws IOException, AsterixException {
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        int numFields = record.getType().getFieldNames().length;
-        for (int pos = 0; pos < numFields; pos++) {
-            fieldValue.reset();
-            IAObject obj = record.getValueByPos(pos);
-            writeObject(obj, fieldValue.getDataOutput());
-            recordBuilder.addField(pos, fieldValue);
-        }
-        recordBuilder.write(dataOutput, true);
-    }
-
-    private void writeObject(IAObject obj, DataOutput dataOutput) throws IOException, AsterixException {
-        switch (obj.getType().getTypeTag()) {
-            case RECORD: {
-                IARecordBuilder recordBuilder = new RecordBuilder();
-                recordBuilder.reset((ARecordType) obj.getType());
-                recordBuilder.init();
-                writeRecord((AMutableRecord) obj, dataOutput, recordBuilder);
-                break;
-            }
-
-            case ORDEREDLIST: {
-                OrderedListBuilder listBuilder = new OrderedListBuilder();
-                listBuilder.reset((AOrderedListType) ((AMutableOrderedList) obj).getType());
-                IACursor cursor = ((AMutableOrderedList) obj).getCursor();
-                ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
-                while (cursor.next()) {
-                    listItemValue.reset();
-                    IAObject item = cursor.get();
-                    writeObject(item, listItemValue.getDataOutput());
-                    listBuilder.addItem(listItemValue);
-                }
-                listBuilder.write(dataOutput, true);
-                break;
-            }
-
-            case UNORDEREDLIST: {
-                UnorderedListBuilder listBuilder = new UnorderedListBuilder();
-                listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
-                IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
-                ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
-                while (cursor.next()) {
-                    listItemValue.reset();
-                    IAObject item = cursor.get();
-                    writeObject(item, listItemValue.getDataOutput());
-                    listBuilder.addItem(listItemValue);
-                }
-                listBuilder.write(dataOutput, true);
-                break;
-            }
-
-            default:
-                AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(obj.getType()).serialize(obj,
-                        dataOutput);
-                break;
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
deleted file mode 100644
index ff9af0c..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public abstract class FileSystemBasedAdapter implements IDatasourceAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
-
-    public abstract InputStream getInputStream(int partition) throws IOException;
-
-    protected final ITupleParserFactory parserFactory;
-    protected ITupleParser tupleParser;
-    protected final IAType sourceDatatype;
-    protected IHyracksTaskContext ctx;
-
-    public FileSystemBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        this.parserFactory = parserFactory;
-        this.sourceDatatype = sourceDatatype;
-        this.ctx = ctx;
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        tupleParser = parserFactory.createTupleParser(ctx);
-        InputStream in = getInputStream(partition);
-        tupleParser.parse(in, writer);
-    }
-
-    public String getFilename(int partition) {
-        return null;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java
new file mode 100644
index 0000000..74e98dd
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataset.adapter;
+
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.hyracks.api.comm.IFrameWriter;
+
+public class GenericAdapter implements IDataSourceAdapter {
+
+    private static final long serialVersionUID = 1L;
+    private final IDataFlowController controller;
+
+    public GenericAdapter(IDataFlowController controller) {
+        this.controller = controller;
+    }
+
+    @Override
+    public void start(int partition, IFrameWriter writer) throws Exception {
+        controller.start(writer);
+    }
+
+    @Override
+    public boolean stop() throws Exception {
+        return controller.stop();
+    }
+
+    @Override
+    public boolean handleException(Throwable e) {
+        return controller.handleException(e);
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSAdapter.java
deleted file mode 100644
index 5f1b1ae..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSAdapter.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.input.GenericFileAwareRecordReader;
-import org.apache.asterix.external.indexing.input.GenericRecordReader;
-import org.apache.asterix.external.indexing.input.TextualDataReader;
-import org.apache.asterix.external.indexing.input.TextualFullScanDataReader;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * Provides functionality for fetching external data stored in an HDFS instance.
- */
-
-public class HDFSAdapter extends FileSystemBasedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private transient String[] readSchedule;
-    private transient boolean executed[];
-    private transient InputSplit[] inputSplits;
-    private transient JobConf conf;
-    private transient String nodeName;
-    private transient List<ExternalFile> files;
-    private transient Map<String, String> configuration;
-
-    public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-            String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx,
-            Map<String, String> configuration, List<ExternalFile> files) throws HyracksDataException {
-        super(parserFactory, atype, ctx);
-        this.readSchedule = readSchedule;
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.conf = conf;
-        this.nodeName = nodeName;
-        this.files = files;
-        this.configuration = configuration;
-    }
-
-    /*
-     * The method below was modified to take care of the following
-     * 1. when target files are not null, it generates a file aware input stream that validate against the files
-     * 2. if the data is binary, it returns a generic reader
-     */
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        if ((conf.getInputFormat() instanceof TextInputFormat
-                || conf.getInputFormat() instanceof SequenceFileInputFormat)
-                && (AsterixTupleParserFactory.FORMAT_ADM
-                        .equalsIgnoreCase(configuration.get(AsterixTupleParserFactory.KEY_FORMAT))
-                        || AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT
-                                .equalsIgnoreCase(configuration.get(AsterixTupleParserFactory.KEY_FORMAT)))) {
-            if (files != null) {
-                return new TextualDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
-            } else {
-                return new TextualFullScanDataReader(executed, inputSplits, readSchedule, nodeName, conf);
-            }
-        } else {
-            if (files != null) {
-                return new GenericFileAwareRecordReader(inputSplits, readSchedule, nodeName, conf, executed, files);
-            } else {
-                return new GenericRecordReader(inputSplits, readSchedule, nodeName, conf, executed);
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
deleted file mode 100644
index 92a049d0..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
-
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.input.GenericFileAwareRecordReader;
-import org.apache.asterix.external.indexing.input.RCFileDataReader;
-import org.apache.asterix.external.indexing.input.TextualDataReader;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public class HDFSIndexingAdapter extends FileSystemBasedAdapter {
-
-    private static final long serialVersionUID = 1L;
-    private transient String[] readSchedule;
-    private transient boolean executed[];
-    private transient InputSplit[] inputSplits;
-    private transient JobConf conf;
-    private final List<ExternalFile> files;
-    private transient String nodeName;
-    // file input-format <text, seq, rc>
-    private String inputFormat;
-    // content format <adm, delimited-text, binary>
-    private String format;
-
-    public HDFSIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits,
-            JobConf conf, AlgebricksPartitionConstraint clusterLocations, List<ExternalFile> files,
-            ITupleParserFactory parserFactory, IHyracksTaskContext ctx, String nodeName, String inputFormat,
-            String format) throws IOException {
-        super(parserFactory, atype, ctx);
-        this.nodeName = nodeName;
-        this.readSchedule = readSchedule;
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.conf = conf;
-        this.files = files;
-        this.inputFormat = inputFormat;
-        this.format = format;
-    }
-
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
-            return new RCFileDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
-        } else if (format.equals(AsterixTupleParserFactory.FORMAT_ADM)
-                || format.equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
-            return new TextualDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
-        } else {
-            return new GenericFileAwareRecordReader(inputSplits, readSchedule, nodeName, conf, executed, files);
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HiveAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HiveAdapter.java
deleted file mode 100644
index 1b8024d..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/HiveAdapter.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
- */
-public class HiveAdapter extends FileSystemBasedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private HDFSAdapter hdfsAdapter;
-
-    public HiveAdapter(IAType atype, HDFSAdapter hdfsAdapter, ITupleParserFactory parserFactory, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        super(parserFactory, atype, ctx);
-        this.hdfsAdapter = hdfsAdapter;
-    }
-
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        return hdfsAdapter.getInputStream(partition);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IControlledAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IControlledAdapter.java
deleted file mode 100644
index e71f10c..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IControlledAdapter.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- *
- * @author alamouda
- *
- */
-public interface IControlledAdapter extends Serializable {
-    
-    /**
-     * 
-     * @param ctx
-     * @param recordDescriptors 
-     * @throws Exception
-     */
-    public void initialize(IHyracksTaskContext ctx, INullWriterFactory iNullWriterFactory) throws Exception;
-
-    /**
-     * 
-     * @param buffer
-     * @param writer
-     * @throws HyracksDataException
-     */
-    public void nextFrame(ByteBuffer buffer, IFrameWriter writer) throws Exception;
-
-    /**
-     * 
-     * @param writer
-     * @throws HyracksDataException
-     */
-    public void close(IFrameWriter writer) throws Exception;
-    
-    /**
-     * Gives the adapter a chance to clean up
-     * @param writer
-     * @throws HyracksDataException
-     */
-    public void fail() throws Exception;
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClient.java
deleted file mode 100644
index 6377960..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClient.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-
-public interface IFeedClient {
-
-    public enum InflowState {
-        NO_MORE_DATA,
-        DATA_AVAILABLE,
-        DATA_NOT_AVAILABLE
-    }
-
-    /**
-     * Writes the next fetched tuple into the provided instance of DatatOutput. Invocation of this method blocks until
-     * a new tuple has been written or the specified time has expired.
-     * 
-     * @param dataOutput
-     *            The receiving channel for the feed client to write ADM records to.
-     * @param timeout
-     *            Threshold time (expressed in seconds) for the next tuple to be obtained from the external source.
-     * @return
-     * @throws AsterixException
-     */
-    public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException;
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClientFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClientFactory.java
deleted file mode 100644
index b175518..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IFeedClientFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public interface IFeedClientFactory {
-
-    public IPullBasedFeedClient createFeedClient(IHyracksTaskContext ctx, Map<String, String> configuration)
-            throws Exception;
-
-    public ARecordType getRecordType() throws AsterixException;
-
-    public FeedClientType getFeedClientType();
-
-    public enum FeedClientType {
-        GENERIC,
-        TYPED
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IPullBasedFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IPullBasedFeedClient.java
deleted file mode 100644
index f6c9dad..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/IPullBasedFeedClient.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-
-public interface IPullBasedFeedClient {
-
-    public enum InflowState {
-        NO_MORE_DATA,
-        DATA_AVAILABLE,
-        DATA_NOT_AVAILABLE
-    }
-
-    /**
-     * Writes the next fetched tuple into the provided instance of DatatOutput. Invocation of this method blocks until
-     * a new tuple has been written or the specified time has expired.
-     * 
-     * @param dataOutput
-     *            The receiving channel for the feed client to write ADM records to.
-     * @param timeout
-     *            Threshold time (expressed in seconds) for the next tuple to be obtained from the externa source.
-     * @return
-     * @throws AsterixException
-     */
-    public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException;
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
new file mode 100644
index 0000000..ba6f83c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.dataset.adapter;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.asterix.external.indexing.RecordIdReader;
+import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.INullWriter;
+import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public final class LookupAdapter<T> implements IFrameWriter {
+
+    private boolean propagateInput;
+    private int[] propagatedFields;
+    private boolean retainNull;
+    private ArrayTupleBuilder tb;
+    private FrameTupleAppender appender;
+    private IRecordDataParser<T> dataParser;
+    private ILookupRecordReader<? extends T> recordReader;
+    private RecordIdReader ridReader;
+    private FrameTupleAccessor tupleAccessor;
+    private IFrameWriter writer;
+    private FrameTupleReference frameTuple;
+    private ArrayTupleBuilder nullTupleBuild;
+
+    public LookupAdapter(IRecordDataParser<T> dataParser, ILookupRecordReader<? extends T> recordReader,
+            RecordDescriptor inRecDesc, RecordIdReader ridReader, boolean propagateInput, int[] propagatedFields,
+            boolean retainNull, INullWriterFactory iNullWriterFactory, IHyracksTaskContext ctx, IFrameWriter writer)
+                    throws HyracksDataException {
+        this.dataParser = dataParser;
+        this.recordReader = recordReader;
+        this.propagateInput = propagateInput;
+        this.propagatedFields = propagatedFields;
+        this.retainNull = retainNull;
+        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
+        this.ridReader = ridReader;
+        ridReader.set(tupleAccessor, inRecDesc);
+        configurePropagation(iNullWriterFactory);
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
+        this.writer = writer;
+    }
+
+    private void configurePropagation(INullWriterFactory iNullWriterFactory) {
+        if (propagateInput) {
+            tb = new ArrayTupleBuilder(propagatedFields.length + 1);
+            frameTuple = new FrameTupleReference();
+        } else {
+            tb = new ArrayTupleBuilder(1);
+        }
+        if (retainNull) {
+            INullWriter nullWriter = iNullWriterFactory.createNullWriter();
+            nullTupleBuild = new ArrayTupleBuilder(1);
+            DataOutput out = nullTupleBuild.getDataOutput();
+            try {
+                nullWriter.writeNull(out);
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        } else {
+            nullTupleBuild = null;
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        try {
+            recordReader.fail();
+        } catch (Throwable th) {
+            throw new HyracksDataException(th);
+        } finally {
+            writer.fail();
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        try {
+            tupleAccessor.reset(buffer);
+            int tupleIndex = 0;
+            int tupleCount = tupleAccessor.getTupleCount();
+            while (tupleIndex < tupleCount) {
+                IRawRecord<? extends T> record = null;
+                RecordId rid = ridReader.read(tupleIndex);
+                if (rid != null) {
+                    record = recordReader.read(rid);
+                }
+                tb.reset();
+                if (propagateInput) {
+                    propagate(tupleIndex);
+                }
+                if (record != null) {
+                    dataParser.parse(record, tb.getDataOutput());
+                    tb.addFieldEndOffset();
+                    DataflowUtils.addTupleToFrame(appender, tb, writer);
+                } else if (retainNull) {
+                    tb.getDataOutput().write(nullTupleBuild.getByteArray());
+                    tb.addFieldEndOffset();
+                    DataflowUtils.addTupleToFrame(appender, tb, writer);
+                }
+                tupleIndex++;
+            }
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private void propagate(int idx) throws IOException {
+        frameTuple.reset(tupleAccessor, idx);
+        for (int i = 0; i < propagatedFields.length; i++) {
+            tb.getDataOutput().write(frameTuple.getFieldData(propagatedFields[i]),
+                    frameTuple.getFieldStart(propagatedFields[i]), frameTuple.getFieldLength(propagatedFields[i]));
+            tb.addFieldEndOffset();
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            appender.flush(writer, true);
+        } finally {
+            writer.close();
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/NCFileSystemAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/NCFileSystemAdapter.java
deleted file mode 100644
index 64c62f7..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/NCFileSystemAdapter.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.FileSplit;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * Factory class for creating an instance of NCFileSystemAdapter. An
- * NCFileSystemAdapter reads external data residing on the local file system of
- * an NC.
- */
-public class NCFileSystemAdapter extends FileSystemBasedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private final FileSplit[] fileSplits;
-
-    public NCFileSystemAdapter(FileSplit[] fileSplits, ITupleParserFactory parserFactory, IAType atype,
-            IHyracksTaskContext ctx) throws HyracksDataException {
-        super(parserFactory, atype, ctx);
-        this.fileSplits = fileSplits;
-    }
-
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        FileSplit split = fileSplits[partition];
-        File inputFile = split.getLocalFile().getFile();
-        InputStream in;
-        try {
-            in = new FileInputStream(inputFile);
-            return in;
-        } catch (FileNotFoundException e) {
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public String getFilename(int partition) {
-        final FileSplit fileSplit = fileSplits[partition];
-        return fileSplit.getNodeName() + ":" + fileSplit.getLocalFile().getFile().getPath();
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAdapter.java
deleted file mode 100644
index d6b4ba7..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAdapter.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.dataset.adapter.IPullBasedFeedClient.InflowState;
-import org.apache.asterix.external.feeds.FeedPolicyEnforcer;
-import org.apache.asterix.external.feeds.IPullBasedFeedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-
-/**
- * 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 implements IPullBasedFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(PullBasedAdapter.class.getName());
-    private static final int timeout = 5; // seconds
-
-    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 IFrame frame;
-    private long tupleCount = 0;
-    private final IHyracksTaskContext ctx;
-    private int frameTupleCount = 0;
-
-    protected FeedPolicyEnforcer policyEnforcer;
-
-    public FeedPolicyEnforcer getPolicyEnforcer() {
-        return policyEnforcer;
-    }
-
-    public void setFeedPolicyEnforcer(FeedPolicyEnforcer policyEnforcer) {
-        this.policyEnforcer = policyEnforcer;
-    }
-
-    public abstract IPullBasedFeedClient getFeedClient(int partition) throws Exception;
-
-    public PullBasedAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) {
-        this.ctx = ctx;
-        this.configuration = configuration;
-    }
-
-    public long getIngestedRecordsCount() {
-        return tupleCount;
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        frame = new VSizeFrame(ctx);
-        appender = new FrameTupleAppender(frame);
-
-        pullBasedFeedClient = getFeedClient(partition);
-        InflowState inflowState = null;
-
-        while (continueIngestion) {
-            tupleBuilder.reset();
-            try {
-                // blocking call
-                inflowState = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput(), timeout);
-                switch (inflowState) {
-                    case DATA_AVAILABLE:
-                        tupleBuilder.addFieldEndOffset();
-                        appendTupleToFrame(writer);
-                        frameTupleCount++;
-                        break;
-                    case NO_MORE_DATA:
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("Reached end of feed");
-                        }
-                        appender.flush(writer, true);
-                        tupleCount += frameTupleCount;
-                        frameTupleCount = 0;
-                        continueIngestion = false;
-                        break;
-                    case DATA_NOT_AVAILABLE:
-                        if (frameTupleCount > 0) {
-                            appender.flush(writer, true);
-                            tupleCount += frameTupleCount;
-                            frameTupleCount = 0;
-                        }
-                        if (LOGGER.isLoggable(Level.WARNING)) {
-                            LOGGER.warning("Timed out on obtaining data from pull based adapter. Trying again!");
-                        }
-                        break;
-                }
-
-            } catch (Exception failureException) {
-                try {
-                    failureException.printStackTrace();
-                    boolean continueIngestion = policyEnforcer.continueIngestionPostSoftwareFailure(failureException);
-                    if (continueIngestion) {
-                        tupleBuilder.reset();
-                        continue;
-                    } else {
-                        throw failureException;
-                    }
-                } catch (Exception recoveryException) {
-                    throw new Exception(recoveryException);
-                }
-            }
-        }
-    }
-
-    private void appendTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                tupleBuilder.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                    tupleBuilder.getSize())) {
-                throw new IllegalStateException();
-            }
-        }
-    }
-
-    /**
-     * Discontinue the ingestion of data and end the feed.
-     * 
-     * @throws Exception
-     */
-    public void stop() throws Exception {
-        continueIngestion = false;
-    }
-
-    public Map<String, String> getConfiguration() {
-        return configuration;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureFeedClient.java
deleted file mode 100644
index 985399b..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureFeedClient.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-
-import com.microsoft.windowsazure.services.core.storage.CloudStorageAccount;
-import com.microsoft.windowsazure.services.table.client.CloudTableClient;
-import com.microsoft.windowsazure.services.table.client.TableConstants;
-import com.microsoft.windowsazure.services.table.client.TableQuery;
-import com.microsoft.windowsazure.services.table.client.TableQuery.Operators;
-import com.microsoft.windowsazure.services.table.client.TableQuery.QueryComparisons;
-import com.microsoft.windowsazure.services.table.client.TableServiceEntity;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.library.java.JObjects.ByteArrayAccessibleInputStream;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
-import org.apache.asterix.runtime.operators.file.ADMDataParser;
-
-public class PullBasedAzureFeedClient implements IPullBasedFeedClient {
-    private static final Logger LOGGER = Logger.getLogger(PullBasedAzureFeedClient.class.getName());
-
-    private final String tableName;
-    private final ARecordType outputType;
-    private final CloudTableClient ctc;
-    private final TableQuery<? extends TableServiceEntity> tableQuery;
-    private Iterator<? extends TableServiceEntity> entityIt;
-
-    private final Pattern arrayPattern = Pattern.compile("\\[(?<vals>.*)\\]");
-    private final Pattern int32Pattern = Pattern.compile(":(?<int>\\d+)(,|})");
-    private final Pattern doubleWithEndingZeroPattern = Pattern.compile("\\d+\\.(?<zero>0)(,|})");
-
-    private final ResettableByteArrayOutputStream rbaos;
-    private final DataOutputStream dos;
-    private final ADMDataParser adp;
-    private final ByteArrayAccessibleInputStream baais;
-
-    public PullBasedAzureFeedClient(CloudStorageAccount csa, ARecordType outputType, String tableName, String lowKey,
-            String highKey) throws AsterixException {
-        this.tableName = tableName;
-        this.outputType = outputType;
-        this.tableQuery = configureTableQuery(tableName, lowKey, highKey);
-        this.ctc = csa.createCloudTableClient();
-        rbaos = new ResettableByteArrayOutputStream();
-        dos = new DataOutputStream(rbaos);
-        baais = new ByteArrayAccessibleInputStream(rbaos.getByteArray(), 0, 0);
-        adp = new ADMDataParser();
-        adp.initialize(baais, outputType, false);
-    }
-
-    private TableQuery<? extends TableServiceEntity> configureTableQuery(String tableName, String lowKey, String highKey) {
-        TableQuery<? extends TableServiceEntity> baseTQ = TableQuery.from(tableName, classFromString(tableName));
-        if (lowKey != null && highKey != null) {
-            String lowKeyPredicate = TableQuery.generateFilterCondition(TableConstants.PARTITION_KEY,
-                    QueryComparisons.GREATER_THAN_OR_EQUAL, lowKey);
-            String highKeyPredicate = TableQuery.generateFilterCondition(TableConstants.PARTITION_KEY,
-                    QueryComparisons.LESS_THAN_OR_EQUAL, highKey);
-            String partitionPredicate = TableQuery.combineFilters(lowKeyPredicate, Operators.AND, highKeyPredicate);
-            return baseTQ.where(partitionPredicate);
-        }
-
-        return baseTQ;
-    }
-
-    private Class<? extends TableServiceEntity> classFromString(String tableName) {
-        return tableName.equals("Postings") ? AzureTweetEntity.class : AzureTweetMetadataEntity.class;
-    }
-
-    @Override
-    public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException {
-        if (entityIt == null) {
-            entityIt = ctc.execute(tableQuery).iterator();
-        }
-
-        boolean moreTweets = entityIt.hasNext();
-        if (moreTweets) {
-            String json = null;
-            try {
-                json = getJSONString();
-                byte[] jsonBytes = json.getBytes(StandardCharsets.UTF_8);
-                rbaos.reset();
-                dos.write(jsonBytes, 0, jsonBytes.length);
-                dos.flush();
-                baais.setContent(rbaos.getByteArray(), 0, jsonBytes.length);
-                adp.initialize(baais, outputType, false);
-                adp.parse(dataOutput);
-            } catch (Exception e) {
-                if (json != null) {
-                    if (LOGGER.isLoggable(Level.SEVERE)) {
-                        LOGGER.severe("Record in error: " + json);
-                    }
-                }
-                e.printStackTrace();
-                throw new AsterixException(e);
-            }
-        }
-        return moreTweets ? InflowState.DATA_AVAILABLE : InflowState.NO_MORE_DATA;
-    }
-
-    private String getJSONString() throws JSONException {
-        if (tableName.equals("Postings")) {
-            AzureTweetEntity tweet = (AzureTweetEntity) entityIt.next();
-            JSONObject tjo = new JSONObject(tweet.getJSON().toString());
-            tjo.put("posting_id", tweet.getRowKey());
-            tjo.put("user_id", tweet.getPartitionKey());
-            tjo.remove("id");
-            JSONObject utjo = tjo.getJSONObject("user");
-            utjo.remove("id");
-            tjo.put("user", utjo);
-            return tjo.toString();
-        } else if (tableName.equals("PostingMetadata")) {
-            AzureTweetMetadataEntity tweetMD = (AzureTweetMetadataEntity) entityIt.next();
-            JSONObject tmdjo = new JSONObject();
-            tmdjo.put("posting_id", tweetMD.getRowKey());
-            tmdjo.put("user_id", tweetMD.getPartitionKey());
-            tmdjo.put("created_at", stripTillColon(tweetMD.getCreationTimestamp()).replaceAll("\"", ""));
-            tmdjo.put("posting_type", stripTillColon(tweetMD.getPostingType()));
-            List<Integer> productIdList = Arrays.asList(extractArray(tweetMD.getProductId()));
-            tmdjo.put("product_id", productIdList);
-            if (tweetMD.getEthnicity() != null) {
-                tmdjo.put("ethnicity", new JSONObject(stripTillColon(tweetMD.getEthnicity())));
-            }
-            if (tweetMD.getGender() != null) {
-                tmdjo.put("gender", new JSONObject(stripTillColon(tweetMD.getGender())));
-            }
-            if (tweetMD.getLocation() != null) {
-                String locStr = stripTillColon(tweetMD.getLocation());
-                Matcher m = int32Pattern.matcher(locStr);
-                while (m.find()) {
-                    locStr = locStr.replace(m.group("int"), m.group("int") + ".01");
-                }
-                m = doubleWithEndingZeroPattern.matcher(locStr);
-                while (m.find()) {
-                    locStr = locStr.replace(m.group("zero"), "01");
-                }
-                tmdjo.put("location", new JSONObject(locStr));
-            }
-            if (tweetMD.getSentiment() != null) {
-                tmdjo.put("sentiment", stripTillColon(tweetMD.getSentiment()));
-            }
-            return tmdjo.toString();
-        } else {
-            throw new IllegalArgumentException();
-        }
-    }
-
-    private String stripTillColon(String str) {
-        return str.substring(str.indexOf(':') + 1);
-    }
-
-    private Integer[] extractArray(String str) {
-        Matcher m = arrayPattern.matcher(str);
-        m.find();
-        String[] stringNums = m.group("vals").replaceAll("\\s", "").split(",");
-        Integer[] nums = new Integer[stringNums.length];
-        for (int i = 0; i < nums.length; ++i) {
-            nums[i] = Integer.parseInt(stringNums[i]);
-        }
-        return nums;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java
deleted file mode 100644
index e8cacde..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.net.URISyntaxException;
-import java.security.InvalidKeyException;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import com.microsoft.windowsazure.services.core.storage.CloudStorageAccount;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class PullBasedAzureTwitterAdapter extends PullBasedAdapter implements IDatasourceAdapter {
-    private static final Logger LOGGER = Logger.getLogger(PullBasedAzureTwitterAdapter.class.getName());
-
-    private static final long serialVersionUID = 1L;
-
-    private final CloudStorageAccount csa;
-    private final String connectionString;
-    private final String azureAccountName;
-    private final String azureAccountKey;
-    private final ARecordType outputType;
-    private final String tableName;
-    private final boolean partitioned;
-
-    private String[] lowKeys;
-    private String[] highKeys;
-
-    public PullBasedAzureTwitterAdapter(String accountName, String accountKey, String tableName, String[] partitions,
-            Map<String, String> configuration, IHyracksTaskContext ctx, ARecordType outputType) throws AsterixException {
-        super(configuration, ctx);
-        this.outputType = outputType;
-        if (partitions != null) {
-            partitioned = true;
-            configurePartitions(partitions);
-        } else {
-            partitioned = false;
-        }
-        this.azureAccountName = accountName;
-        this.azureAccountKey = accountKey;
-        this.tableName = tableName;
-
-        connectionString = "DefaultEndpointsProtocol=http;" + "AccountName=" + azureAccountName + ";AccountKey="
-                + azureAccountKey + ";";
-        try {
-            csa = CloudStorageAccount.parse(connectionString);
-        } catch (InvalidKeyException | URISyntaxException e) {
-            throw new AsterixException("You must specify a valid Azure account name and key", e);
-        }
-    }
-
-    private void configurePartitions(String[] partitions) {
-        lowKeys = new String[partitions.length];
-        highKeys = new String[partitions.length];
-        for (int i = 0; i < partitions.length; ++i) {
-            String[] loHi = partitions[i].split(":");
-            lowKeys[i] = loHi[0];
-            highKeys[i] = loHi[1];
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Partition " + i + " configured for keys " + lowKeys[i] + " to " + highKeys[i]);
-            }
-        }
-    }
-
-    @Override
-    public IPullBasedFeedClient getFeedClient(int partition) throws Exception {
-        if (partitioned) {
-            return new PullBasedAzureFeedClient(csa, outputType, tableName, lowKeys[partition], highKeys[partition]);
-        }
-        return new PullBasedAzureFeedClient(csa, outputType, tableName, null, null);
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PULL;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return false;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
deleted file mode 100644
index 90281b7..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.CounterTimerTupleForwardPolicy;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * An adapter that provides the functionality of receiving tweets from the
- * Twitter service in the form of ADM formatted records.
- */
-public class PullBasedTwitterAdapter extends ClientBasedFeedAdapter implements IFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final int DEFAULT_BATCH_SIZE = 5;
-
-    private ARecordType recordType;
-    private PullBasedTwitterFeedClient tweetClient;
-
-    @Override
-    public IFeedClient getFeedClient(int partition) {
-        return tweetClient;
-    }
-
-    public PullBasedTwitterAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
-            throws AsterixException {
-        super(configuration, ctx);
-        tweetClient = new PullBasedTwitterFeedClient(ctx, recordType, this);
-    }
-
-    public ARecordType getAdapterOutputType() {
-        return recordType;
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PULL;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return true;
-    }
-
-    @Override
-    public ITupleForwardPolicy getTupleParserPolicy() {
-        configuration.put(ITupleForwardPolicy.PARSER_POLICY,
-                ITupleForwardPolicy.TupleForwardPolicyType.COUNTER_TIMER_EXPIRED.name());
-        String propValue = configuration.get(CounterTimerTupleForwardPolicy.BATCH_SIZE);
-        if (propValue == null) {
-            configuration.put(CounterTimerTupleForwardPolicy.BATCH_SIZE, "" + DEFAULT_BATCH_SIZE);
-        }
-        return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
deleted file mode 100644
index 8b5e1e1..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.List;
-import java.util.Map;
-
-import twitter4j.Query;
-import twitter4j.QueryResult;
-import twitter4j.Status;
-import twitter4j.Twitter;
-import twitter4j.TwitterException;
-import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.external.util.TweetProcessor;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * An implementation of @see {PullBasedFeedClient} for the Twitter service. The
- * feed client fetches data from Twitter service by sending request at regular
- * (configurable) interval.
- */
-public class PullBasedTwitterFeedClient extends FeedClient {
-
-    private String keywords;
-    private Query query;
-    private Twitter twitter;
-    private int requestInterval = 5; // seconds
-    private QueryResult result;
-
-    private ARecordType recordType;
-    private int nextTweetIndex = 0;
-    private long lastTweetIdReceived = 0;
-    private TweetProcessor tweetProcessor;
-
-    public PullBasedTwitterFeedClient(IHyracksTaskContext ctx, ARecordType recordType, PullBasedTwitterAdapter adapter) {
-        this.twitter = TwitterUtil.getTwitterService(adapter.getConfiguration());
-        this.recordType = recordType;
-        this.tweetProcessor = new TweetProcessor(recordType);
-        this.recordSerDe = new ARecordSerializerDeserializer(recordType);
-        this.mutableRecord = tweetProcessor.getMutableRecord();
-        this.initialize(adapter.getConfiguration());
-    }
-
-    public ARecordType getRecordType() {
-        return recordType;
-    }
-
-    @Override
-    public InflowState retrieveNextRecord() throws Exception {
-        Status tweet;
-        tweet = getNextTweet();
-        if (tweet == null) {
-            return InflowState.DATA_NOT_AVAILABLE;
-        }
-
-        tweetProcessor.processNextTweet(tweet);
-        return InflowState.DATA_AVAILABLE;
-    }
-
-    private void initialize(Map<String, String> params) {
-        this.keywords = (String) params.get(SearchAPIConstants.QUERY);
-        this.requestInterval = Integer.parseInt((String) params.get(SearchAPIConstants.INTERVAL));
-        this.query = new Query(keywords);
-        this.query.setCount(100);
-    }
-
-    private Status getNextTweet() throws TwitterException, InterruptedException {
-        if (result == null || nextTweetIndex >= result.getTweets().size()) {
-            Thread.sleep(1000 * requestInterval);
-            query.setSinceId(lastTweetIdReceived);
-            result = twitter.search(query);
-            nextTweetIndex = 0;
-        }
-        if (result != null && !result.getTweets().isEmpty()) {
-            List<Status> tw = result.getTweets();
-            Status tweet = tw.get(nextTweetIndex++);
-            if (lastTweetIdReceived < tweet.getId()) {
-                lastTweetIdReceived = tweet.getId();
-            }
-            return tweet;
-        } else {
-            return null;
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java
deleted file mode 100644
index 01839d31..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.CounterTimerTupleForwardPolicy;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class PushBasedTwitterAdapter extends ClientBasedFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final int DEFAULT_BATCH_SIZE = 50;
-
-    private PushBasedTwitterFeedClient tweetClient;
-
-    public PushBasedTwitterAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx) throws AsterixException {
-        super(configuration, ctx);
-        this.configuration = configuration;
-        this.tweetClient = new PushBasedTwitterFeedClient(ctx, recordType, this);
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PUSH;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return true;
-    }
-
-    @Override
-    public IFeedClient getFeedClient(int partition) throws Exception {
-        return tweetClient;
-    }
-
-    @Override
-    public ITupleForwardPolicy getTupleParserPolicy() {
-        configuration.put(ITupleForwardPolicy.PARSER_POLICY,
-                ITupleForwardPolicy.TupleForwardPolicyType.COUNTER_TIMER_EXPIRED.name());
-        String propValue = configuration.get(CounterTimerTupleForwardPolicy.BATCH_SIZE);
-        if (propValue == null) {
-            configuration.put(CounterTimerTupleForwardPolicy.BATCH_SIZE, "" + DEFAULT_BATCH_SIZE);
-        }
-        return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java
deleted file mode 100644
index bb40ac9..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.external.util.TweetProcessor;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-import twitter4j.FilterQuery;
-import twitter4j.StallWarning;
-import twitter4j.Status;
-import twitter4j.StatusDeletionNotice;
-import twitter4j.StatusListener;
-import twitter4j.TwitterStream;
-
-/**
- * An implementation of @see {PullBasedFeedClient} for the Twitter service. The
- * feed client fetches data from Twitter service by sending request at regular
- * (configurable) interval.
- */
-public class PushBasedTwitterFeedClient extends FeedClient {
-
-    private ARecordType recordType;
-    private TweetProcessor tweetProcessor;
-    private LinkedBlockingQueue<Status> inputQ;
-
-    public PushBasedTwitterFeedClient(IHyracksTaskContext ctx, ARecordType recordType, PushBasedTwitterAdapter adapter)
-            throws AsterixException {
-        this.recordType = recordType;
-        this.tweetProcessor = new TweetProcessor(recordType);
-        this.recordSerDe = new ARecordSerializerDeserializer(recordType);
-        this.mutableRecord = tweetProcessor.getMutableRecord();
-        this.inputQ = new LinkedBlockingQueue<Status>();
-        TwitterStream twitterStream = TwitterUtil.getTwitterStream(adapter.getConfiguration());
-        twitterStream.addListener(new TweetListener(inputQ));
-        FilterQuery query = TwitterUtil.getFilterQuery(adapter.getConfiguration());
-        if (query != null) {
-            twitterStream.filter(query);
-        } else {
-            twitterStream.sample();
-        }
-    }
-
-    public ARecordType getRecordType() {
-        return recordType;
-    }
-
-    private class TweetListener implements StatusListener {
-
-        private LinkedBlockingQueue<Status> inputQ;
-
-        public TweetListener(LinkedBlockingQueue<Status> inputQ) {
-            this.inputQ = inputQ;
-        }
-
-        @Override
-        public void onStatus(Status tweet) {
-            inputQ.add(tweet);
-        }
-
-        @Override
-        public void onException(Exception arg0) {
-
-        }
-
-        @Override
-        public void onDeletionNotice(StatusDeletionNotice arg0) {
-        }
-
-        @Override
-        public void onScrubGeo(long arg0, long arg1) {
-        }
-
-        @Override
-        public void onStallWarning(StallWarning arg0) {
-        }
-
-        @Override
-        public void onTrackLimitationNotice(int arg0) {
-        }
-    }
-
-    @Override
-    public InflowState retrieveNextRecord() throws Exception {
-        Status tweet = inputQ.take();
-        tweetProcessor.processNextTweet(tweet);
-        return InflowState.DATA_AVAILABLE;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedAdapter.java
deleted file mode 100644
index 69cd82c..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedAdapter.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * RSSFeedAdapter provides the functionality of fetching an RSS based feed.
- */
-public class RSSFeedAdapter extends ClientBasedFeedAdapter implements IFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final String KEY_RSS_URL = "url";
-
-    private List<String> feedURLs = new ArrayList<String>();
-    private String id_prefix = "";
-
-    private IFeedClient rssFeedClient;
-
-    private ARecordType recordType;
-
-    public RSSFeedAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
-            throws AsterixException {
-        super(configuration, ctx);
-        id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
-        this.recordType = recordType;
-        reconfigure(configuration);
-    }
-
-    private void initializeFeedURLs(String rssURLProperty) {
-        feedURLs.clear();
-        String[] feedURLProperty = rssURLProperty.split(",");
-        for (String feedURL : feedURLProperty) {
-            feedURLs.add(feedURL);
-        }
-    }
-
-    protected void reconfigure(Map<String, String> arguments) {
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
-        if (rssURLProperty != null) {
-            initializeFeedURLs(rssURLProperty);
-        }
-    }
-
-    @Override
-    public IFeedClient getFeedClient(int partition) throws Exception {
-        if (rssFeedClient == null) {
-            rssFeedClient = new RSSFeedClient(this, feedURLs.get(partition), id_prefix);
-        }
-        return rssFeedClient;
-    }
-
-    public ARecordType getRecordType() {
-        return recordType;
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PULL;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return false;
-    }
-
-    @Override
-    public ITupleForwardPolicy getTupleParserPolicy() {
-        return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedClient.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedClient.java
deleted file mode 100644
index 0b0d0fb..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/RSSFeedClient.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.dataset.adapter;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-
-import com.sun.syndication.feed.synd.SyndEntryImpl;
-import com.sun.syndication.feed.synd.SyndFeed;
-import com.sun.syndication.fetcher.FeedFetcher;
-import com.sun.syndication.fetcher.FetcherEvent;
-import com.sun.syndication.fetcher.FetcherListener;
-import com.sun.syndication.fetcher.impl.FeedFetcherCache;
-import com.sun.syndication.fetcher.impl.HashMapFeedInfoCache;
-import com.sun.syndication.fetcher.impl.HttpURLFeedFetcher;
-
-import org.apache.asterix.om.base.AMutableRecord;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.ARecordType;
-
-/**
- * An implementation of @see {PullBasedFeedClient} responsible for
- * fetching from an RSS feed source at regular interval.
- */
-@SuppressWarnings("rawtypes")
-public class RSSFeedClient extends FeedClient {
-
-    private long id = 0;
-    private String idPrefix;
-    private boolean feedModified = false;
-
-    private Queue<SyndEntryImpl> rssFeedBuffer = new LinkedList<SyndEntryImpl>();
-
-    IAObject[] mutableFields;
-
-    private final FeedFetcherCache feedInfoCache;
-    private final FeedFetcher fetcher;
-    private final FetcherEventListenerImpl listener;
-    private final URL feedUrl;
-    private ARecordType recordType;
-    String[] tupleFieldValues;
-
-    public boolean isFeedModified() {
-        return feedModified;
-    }
-
-    public void setFeedModified(boolean feedModified) {
-        this.feedModified = feedModified;
-    }
-
-    public RSSFeedClient(RSSFeedAdapter adapter, String feedURL, String id_prefix) throws MalformedURLException {
-        this.idPrefix = id_prefix;
-        this.feedUrl = new URL(feedURL);
-        feedInfoCache = HashMapFeedInfoCache.getInstance();
-        fetcher = new HttpURLFeedFetcher(feedInfoCache);
-        listener = new FetcherEventListenerImpl(this);
-        fetcher.addFetcherEventListener(listener);
-        mutableFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableString(null),
-                new AMutableString(null) };
-        recordType = adapter.getRecordType();
-        mutableRecord = new AMutableRecord(recordType, mutableFields);
-        tupleFieldValues = new String[recordType.getFieldNames().length];
-    }
-
-    @Override
-    public InflowState retrieveNextRecord() throws Exception {
-        SyndEntryImpl feedEntry = getNextRSSFeed();
-        if (feedEntry == null) {
-            return InflowState.DATA_NOT_AVAILABLE;
-        }
-        tupleFieldValues[0] = idPrefix + ":" + id;
-        tupleFieldValues[1] = feedEntry.getTitle();
-        tupleFieldValues[2] = feedEntry.getDescription().getValue();
-        tupleFieldValues[3] = feedEntry.getLink();
-        int numFields = recordType.getFieldNames().length;
-        for (int i = 0; i < numFields; i++) {
-            ((AMutableString) mutableFields[i]).setValue(tupleFieldValues[i]);
-            mutableRecord.setValueAtPos(i, mutableFields[i]);
-        }
-        id++;
-        return InflowState.DATA_AVAILABLE;
-    }
-
-    private SyndEntryImpl getNextRSSFeed() throws Exception {
-        if (rssFeedBuffer.isEmpty()) {
-            fetchFeed();
-        }
-        if (rssFeedBuffer.isEmpty()) {
-            return null;
-        } else {
-            return rssFeedBuffer.remove();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    private void fetchFeed() {
-        try {
-            // Retrieve the feed.
-            // We will get a Feed Polled Event and then a
-            // Feed Retrieved event (assuming the feed is valid)
-            SyndFeed feed = fetcher.retrieveFeed(feedUrl);
-            if (feedModified) {
-                System.err.println(feedUrl + " retrieved");
-                System.err.println(feedUrl + " has a title: " + feed.getTitle() + " and contains "
-                        + feed.getEntries().size() + " entries.");
-
-                List fetchedFeeds = feed.getEntries();
-                rssFeedBuffer.addAll(fetchedFeeds);
-            }
-        } catch (Exception ex) {
-            System.out.println("ERROR: " + ex.getMessage());
-            ex.printStackTrace();
-        }
-    }
-
-}
-
-class FetcherEventListenerImpl implements FetcherListener {
-
-    private final IFeedClient feedClient;
-
-    public FetcherEventListenerImpl(IFeedClient feedClient) {
-        this.feedClient = feedClient;
-    }
-
-    /**
-     * @see com.sun.syndication.fetcher.FetcherListener#fetcherEvent(com.sun.syndication.fetcher.FetcherEvent)
-     */
-    public void fetcherEvent(FetcherEvent event) {
-        String eventType = event.getEventType();
-        if (FetcherEvent.EVENT_TYPE_FEED_POLLED.equals(eventType)) {
-            System.err.println("\tEVENT: Feed Polled. URL = " + event.getUrlString());
-        } else if (FetcherEvent.EVENT_TYPE_FEED_RETRIEVED.equals(eventType)) {
-            System.err.println("\tEVENT: Feed Retrieved. URL = " + event.getUrlString());
-            ((RSSFeedClient) feedClient).setFeedModified(true);
-        } else if (FetcherEvent.EVENT_TYPE_FEED_UNCHANGED.equals(eventType)) {
-            System.err.println("\tEVENT: Feed Unchanged. URL = " + event.getUrlString());
-            ((RSSFeedClient) feedClient).setFeedModified(true);
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/StreamBasedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/StreamBasedAdapter.java
index b436177..3f10dc4 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/StreamBasedAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/StreamBasedAdapter.java
@@ -23,7 +23,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -31,7 +31,7 @@
 import org.apache.hyracks.dataflow.std.file.ITupleParser;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public abstract class StreamBasedAdapter implements IDatasourceAdapter {
+public abstract class StreamBasedAdapter implements IDataSourceAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -43,8 +43,8 @@
 
     protected final IAType sourceDatatype;
 
-    public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx, int partition)
-            throws HyracksDataException {
+    public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx,
+            int partition) throws HyracksDataException {
         this.tupleParser = parserFactory.createTupleParser(ctx);
         this.sourceDatatype = sourceDatatype;
     }
@@ -56,7 +56,8 @@
             tupleParser.parse(in, writer);
         } else {
             if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("Could not obtain input stream for parsing from adapter " + this + "[" + partition + "]");
+                LOGGER.warning(
+                        "Could not obtain input stream for parsing from adapter " + this + "[" + partition + "]");
             }
         }
     }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
index 3988f1a..533d119 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
@@ -40,6 +40,16 @@
     private int fileNumber;
     private ExternalFilePendingOp pendingOp;
 
+    public ExternalFile() {
+        this.dataverseName = "";
+        this.datasetName = "";
+        this.fileNumber = -1;
+        this.fileName = "";
+        this.lastModefiedTime = new Date();
+        this.size = 0;
+        this.pendingOp = ExternalFilePendingOp.PENDING_NO_OP;
+    }
+
     public ExternalFile(String dataverseName, String datasetName, int fileNumber, String fileName,
             Date lastModefiedTime, long size, ExternalFilePendingOp pendingOp) {
         this.dataverseName = dataverseName;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
index b10379b..d94db08 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
@@ -24,7 +24,7 @@
 import java.io.Serializable;
 import java.util.Date;
 
-import org.apache.asterix.external.indexing.operators.ExternalLoopkupOperatorDiscriptor;
+import org.apache.asterix.external.operators.ExternalLookupOperatorDescriptor;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADateTime;
 import org.apache.asterix.om.base.AInt64;
@@ -57,7 +57,7 @@
     private final FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
     private static final long serialVersionUID = 1L;
     private ExternalBTreeDataflowHelper indexDataflowHelper;
-    private ExternalLoopkupOperatorDiscriptor opDesc;
+    private ExternalLookupOperatorDescriptor opDesc;
 
     private IHyracksTaskContext ctx;
     private ExternalBTree index;
@@ -72,39 +72,34 @@
     private IIndexCursor fileIndexSearchCursor;
 
     public ExternalFileIndexAccessor(ExternalBTreeDataflowHelper indexDataflowHelper,
-            ExternalLoopkupOperatorDiscriptor opDesc) {
+            ExternalLookupOperatorDescriptor opDesc) {
         this.indexDataflowHelper = indexDataflowHelper;
         this.opDesc = opDesc;
     }
 
-    public void openIndex() throws HyracksDataException {
+    public void open() throws HyracksDataException {
         // Open the index and get the instance
         indexDataflowHelper.open();
         index = (ExternalBTree) indexDataflowHelper.getIndexInstance();
-        try {
-            // Create search key and search predicate objects
-            searchKey = new ArrayTupleReference();
-            searchKeyTupleBuilder = new ArrayTupleBuilder(FilesIndexDescription.FILE_KEY_SIZE);
-            searchKeyTupleBuilder.reset();
-            searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
-            searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
-            searchCmp = BTreeUtils.getSearchMultiComparator(index.getComparatorFactories(), searchKey);
-            searchPredicate = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+        // Create search key and search predicate objects
+        searchKey = new ArrayTupleReference();
+        searchKeyTupleBuilder = new ArrayTupleBuilder(FilesIndexDescription.FILE_KEY_SIZE);
+        searchKeyTupleBuilder.reset();
+        searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
+        searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
+        searchCmp = BTreeUtils.getSearchMultiComparator(index.getComparatorFactories(), searchKey);
+        searchPredicate = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
 
-            // create the accessor  and the cursor using the passed version
-            ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx);
-            fileIndexAccessor = index.createAccessor(searchCallback, indexDataflowHelper.getVersion());
-            fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
-        } catch (Exception e) {
-            indexDataflowHelper.close();
-            throw new HyracksDataException(e);
-        }
+        // create the accessor  and the cursor using the passed version
+        ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
+                .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx);
+        fileIndexAccessor = index.createAccessor(searchCallback, indexDataflowHelper.getVersion());
+        fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
     }
 
-    public void searchForFile(int fileNumber, ExternalFile file) throws Exception {
+    public void lookup(int fileId, ExternalFile file) throws Exception {
         // Set search parameters
-        currentFileNumber.setValue(fileNumber);
+        currentFileNumber.setValue(fileId);
         searchKeyTupleBuilder.reset();
         searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
         searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
@@ -122,14 +117,14 @@
             ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
             DataInput in = new DataInputStream(stream);
             ARecord externalFileRecord = (ARecord) filesIndexDescription.EXTERNAL_FILE_RECORD_SERDE.deserialize(in);
-            setExternalFileFromARecord(externalFileRecord, file);
+            setFile(externalFileRecord, file);
         } else {
             // This should never happen
             throw new HyracksDataException("Was not able to find a file in the files index");
         }
     }
 
-    private void setExternalFileFromARecord(ARecord externalFileRecord, ExternalFile file) {
+    private void setFile(ARecord externalFileRecord, ExternalFile file) {
         file.setFileName(
                 ((AString) externalFileRecord.getValueByPos(FilesIndexDescription.EXTERNAL_FILE_NAME_FIELD_INDEX))
                         .getStringValue());
@@ -140,11 +135,13 @@
                         .getChrononTime())));
     }
 
-    public void closeIndex() throws HyracksDataException {
-        try {
-            fileIndexSearchCursor.close();
-        } finally {
-            indexDataflowHelper.close();
+    public void close() throws HyracksDataException {
+        if (index != null) {
+            try {
+                fileIndexSearchCursor.close();
+            } finally {
+                indexDataflowHelper.close();
+            }
         }
     }
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.java
similarity index 86%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.java
index 9271ebe..fa22179 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.java
@@ -16,22 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.dataflow;
+package org.apache.asterix.external.indexing;
 
 import java.io.IOException;
 
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AMutableDateTime;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -50,11 +46,11 @@
     private AMutableInt64 aInt64 = new AMutableInt64(0);
     private AMutableString aString = new AMutableString(null);
     private AMutableDateTime aDateTime = new AMutableDateTime(0);
-    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+    private ISerializerDeserializer<IAObject> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ASTRING);
-    private ISerializerDeserializer<ADateTime> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+    private ISerializerDeserializer<IAObject> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ADATETIME);
-    private ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE
+    private ISerializerDeserializer<IAObject> longSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.AINT64);
     private ArrayTupleReference tuple = new ArrayTupleReference();
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
new file mode 100644
index 0000000..932aece
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class FileOffsetIndexer implements IExternalIndexer {
+
+    private static final long serialVersionUID = 1L;
+    public static final int NUM_OF_FIELDS = 2;
+    protected AMutableInt32 fileNumber = new AMutableInt32(0);
+    protected AMutableInt64 offset = new AMutableInt64(0);
+    protected RecordReader<?, Writable> recordReader;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<IAObject> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<IAObject> longSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT64);
+
+    @Override
+    public void reset(IRecordReader<?> reader) throws IOException {
+        //TODO: Make it more generic since we can't assume it is always going to be HDFS records.
+        @SuppressWarnings("unchecked")
+        HDFSRecordReader<?, Writable> hdfsReader = (HDFSRecordReader<?, Writable>) reader;
+        fileNumber.setValue(hdfsReader.getSnapshot().get(hdfsReader.getCurrentSplitIndex()).getFileNumber());
+        recordReader = hdfsReader.getReader();
+        offset.setValue(recordReader.getPos());
+    }
+
+    @Override
+    public void index(ArrayTupleBuilder tb) throws IOException {
+        tb.addField(intSerde, fileNumber);
+        tb.addField(longSerde, offset);
+        // Get position for next index(tb) call
+        offset.setValue(recordReader.getPos());
+    }
+
+    @Override
+    public int getNumberOfFields() {
+        return NUM_OF_FIELDS;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IndexingScheduler.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
similarity index 99%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IndexingScheduler.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
index 2a51380..870a6df 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IndexingScheduler.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.dataflow;
+package org.apache.asterix.external.indexing;
 
 import java.io.IOException;
 import java.net.InetAddress;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java
new file mode 100644
index 0000000..07d09db
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+import org.apache.asterix.om.base.AInt32;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RCRecordIdReader extends RecordIdReader {
+
+    public RCRecordIdReader(int[] ridFields) {
+        super(ridFields);
+    }
+
+    @Override
+    public RecordId read(int index) throws HyracksDataException {
+        if (super.read(index) == null) {
+            return null;
+        }
+        // Get row number
+        bbis.setByteBuffer(frameBuffer, tupleStartOffset
+                + tupleAccessor.getFieldStartOffset(index, ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]));
+        rid.setRow(
+                ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]].deserialize(dis))
+                        .getIntegerValue());
+        return rid;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
new file mode 100644
index 0000000..14235c00
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class RecordColumnarIndexer implements IExternalIndexer {
+
+    private static final long serialVersionUID = 1L;
+    public static final int NUM_OF_FIELDS = 3;
+    protected AMutableInt32 fileNumber = new AMutableInt32(0);
+    protected AMutableInt64 offset = new AMutableInt64(0);
+    protected long nextOffset;
+    protected AMutableInt32 rowNumber = new AMutableInt32(0);
+    protected RecordReader<?, Writable> recordReader;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<IAObject> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<IAObject> longSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT64);
+
+    @Override
+    public void reset(IRecordReader<?> reader) throws IOException {
+        //TODO: Make this more generic. right now, it works because we only index hdfs files.
+        @SuppressWarnings("unchecked")
+        HDFSRecordReader<?, Writable> hdfsReader = (HDFSRecordReader<?, Writable>) reader;
+        fileNumber.setValue(hdfsReader.getSnapshot().get(hdfsReader.getCurrentSplitIndex()).getFileNumber());
+        recordReader = hdfsReader.getReader();
+        offset.setValue(recordReader.getPos());
+        nextOffset = offset.getLongValue();
+        rowNumber.setValue(0);
+    }
+
+    @Override
+    public void index(ArrayTupleBuilder tb) throws IOException {
+        if (recordReader.getPos() != nextOffset) {
+            // start of a new group
+            offset.setValue(nextOffset);
+            nextOffset = recordReader.getPos();
+            rowNumber.setValue(0);
+        }
+        tb.addField(intSerde, fileNumber);
+        tb.addField(longSerde, offset);
+        tb.addField(intSerde, rowNumber);
+        rowNumber.setValue(rowNumber.getIntegerValue() + 1);
+    }
+
+    @Override
+    public int getNumberOfFields() {
+        return NUM_OF_FIELDS;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.java
new file mode 100644
index 0000000..9027101
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+public class RecordId {
+    public static enum RecordIdType {
+        OFFSET,
+        RC
+    }
+
+    private int fileId;
+    private long offset;
+    private int row;
+
+    public int getFileId() {
+        return fileId;
+    }
+
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
+
+    public long getOffset() {
+        return offset;
+    }
+
+    public void setOffset(long offset) {
+        this.offset = offset;
+    }
+
+    public int getRow() {
+        return row;
+    }
+
+    public void setRow(int row) {
+        this.row = row;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.java
new file mode 100644
index 0000000..2b4cc9c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class RecordIdReader {
+
+    private final static byte nullByte = ATypeTag.NULL.serialize();
+    protected FrameTupleAccessor tupleAccessor;
+    protected int fieldSlotsLength;
+    protected int[] ridFields;
+    protected RecordId rid;
+    protected RecordDescriptor inRecDesc;
+    protected ByteBufferInputStream bbis;
+    protected DataInputStream dis;
+    protected int tupleStartOffset;
+    protected ByteBuffer frameBuffer;
+
+    public RecordIdReader(int[] ridFields) {
+        this.ridFields = ridFields;
+        this.rid = new RecordId();
+    }
+
+    public void set(FrameTupleAccessor accessor, RecordDescriptor inRecDesc) {
+        this.tupleAccessor = accessor;
+        this.fieldSlotsLength = accessor.getFieldSlotsLength();
+        this.inRecDesc = inRecDesc;
+        this.bbis = new ByteBufferInputStream();
+        this.dis = new DataInputStream(bbis);
+    }
+
+    public RecordId read(int index) throws HyracksDataException {
+        tupleStartOffset = tupleAccessor.getTupleStartOffset(index) + fieldSlotsLength;
+        int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(index,
+                ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
+        frameBuffer = tupleAccessor.getBuffer();
+        if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
+            return null;
+        }
+        // Get file number
+        bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
+        rid.setFileId(
+                ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]].deserialize(dis))
+                        .getIntegerValue());
+        // Get record group offset
+        bbis.setByteBuffer(frameBuffer, tupleStartOffset
+                + tupleAccessor.getFieldStartOffset(index, ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
+        rid.setOffset(((AInt64) inRecDesc.getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
+                .deserialize(dis)).getLongValue());
+        return rid;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java
new file mode 100644
index 0000000..d0bf2ff
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.indexing;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.util.HDFSUtils;
+
+public class RecordIdReaderFactory {
+
+    public static RecordIdReader create(Map<String, String> configuration, int[] ridFields) throws AsterixException {
+        switch (HDFSUtils.getRecordIdType(configuration)) {
+            case OFFSET:
+                return new RecordIdReader(ridFields);
+            case RC:
+                return new RCRecordIdReader(ridFields);
+            default:
+                throw new AsterixException("Unknown Record Id type: " + HDFSUtils.getRecordIdType(configuration));
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
deleted file mode 100644
index 07e09bd..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.DataOutput;
-import java.io.InputStream;
-
-import org.apache.asterix.external.indexing.input.AbstractHDFSReader;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-
-public abstract class AbstractIndexingTupleParser implements ITupleParser{
-
-    protected ArrayTupleBuilder tb;
-    protected DataOutput dos;
-    protected final FrameTupleAppender appender;
-    protected final ARecordType recType;
-    protected final IHyracksCommonContext ctx;
-    protected final IAsterixHDFSRecordParser deserializer;
-    protected final AMutableInt32 aMutableInt = new AMutableInt32(0);
-    protected final AMutableInt64 aMutableLong = new AMutableInt64(0);
-    
-    @SuppressWarnings("rawtypes")
-    protected final ISerializerDeserializer intSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-    @SuppressWarnings("rawtypes")
-    protected final ISerializerDeserializer longSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-    
-    public AbstractIndexingTupleParser(IHyracksCommonContext ctx, ARecordType recType, IAsterixHDFSRecordParser
-            deserializer) throws HyracksDataException {
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        this.recType = recType;
-        this.ctx = ctx;
-        this.deserializer = deserializer;
-    }
-
-    @Override
-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-        AbstractHDFSReader inReader = (AbstractHDFSReader) in;
-        Object record;
-        try {
-            inReader.initialize();
-            record = inReader.readNext();
-            while (record != null) {
-                tb.reset();
-                deserializer.parse(record, tb.getDataOutput());
-                tb.addFieldEndOffset();
-                //append indexing fields
-                appendIndexingData(tb, inReader);
-                addTupleToFrame(writer);
-                record = inReader.readNext();
-            }
-            appender.flush(writer, true);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    protected abstract void appendIndexingData(ArrayTupleBuilder tb,
-            AbstractHDFSReader inReader) throws Exception;
-
-    protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException("Record is too big to fit in a frame");
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
deleted file mode 100644
index c94be6a..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.input.AbstractHDFSLookupInputStream;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.operators.file.IDataParser;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.INullWriter;
-import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-
-/**
- * class implementation for IControlledTupleParser. It provides common
- * functionality involved in parsing data in an external text format (adm or delimited text) in a pipelined manner and packing
- * frames with formed tuples.
- */
-public class AdmOrDelimitedControlledTupleParser implements IControlledTupleParser {
-
-    private ArrayTupleBuilder tb;
-    private transient DataOutput dos;
-    private final FrameTupleAppender appender;
-    protected final ARecordType recType;
-    private IDataParser parser;
-    private boolean propagateInput;
-    private int[] propagatedFields;
-    private int[] ridFields;
-    private RecordDescriptor inRecDesc;
-    private FrameTupleAccessor tupleAccessor;
-    private FrameTupleReference frameTuple;
-    private ByteBufferInputStream bbis;
-    private DataInputStream dis;
-    private AbstractHDFSLookupInputStream in;
-    private boolean parserInitialized = false;
-    private boolean retainNull;
-    protected byte nullByte;
-    protected ArrayTupleBuilder nullTupleBuild;
-
-    public AdmOrDelimitedControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType,
-            AbstractHDFSLookupInputStream in, boolean propagateInput, RecordDescriptor inRecDesc, IDataParser parser,
-            int[] propagatedFields, int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory)
-                    throws HyracksDataException {
-        this.recType = recType;
-        this.in = in;
-        this.propagateInput = propagateInput;
-        this.retainNull = retainNull;
-        this.inRecDesc = inRecDesc;
-        this.propagatedFields = propagatedFields;
-        this.ridFields = ridFields;
-        this.parser = parser;
-        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        if (propagateInput) {
-            tb = new ArrayTupleBuilder(propagatedFields.length + 1);
-        } else {
-            tb = new ArrayTupleBuilder(1);
-        }
-        frameTuple = new FrameTupleReference();
-        dos = tb.getDataOutput();
-        bbis = new ByteBufferInputStream();
-        dis = new DataInputStream(bbis);
-        nullByte = ATypeTag.NULL.serialize();
-        if (retainNull) {
-            INullWriter nullWriter = iNullWriterFactory.createNullWriter();
-            nullTupleBuild = new ArrayTupleBuilder(1);
-            DataOutput out = nullTupleBuild.getDataOutput();
-            try {
-                nullWriter.writeNull(out);
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        } else {
-            nullTupleBuild = null;
-        }
-    }
-
-    @Override
-    public void close(IFrameWriter writer) throws Exception {
-        try {
-            in.close();
-            appender.flush(writer, true);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
-        try {
-            int tupleCount = 0;
-            int tupleIndex = 0;
-            tupleAccessor.reset(frameBuffer);
-            tupleCount = tupleAccessor.getTupleCount();
-            int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
-            // Loop over tuples
-            while (tupleIndex < tupleCount) {
-                boolean found = false;
-                int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
-                int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
-                        ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
-                // Check if null <- for outer join ->
-                if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
-                } else {
-                    // Get file number
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
-                    int fileNumber = ((AInt32) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]].deserialize(dis))
-                                    .getIntegerValue();
-                    // Get record offset
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + tupleAccessor.getFieldStartOffset(tupleIndex,
-                            ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
-                    long recordOffset = ((AInt64) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]].deserialize(dis))
-                                    .getLongValue();
-                    found = in.fetchRecord(fileNumber, recordOffset);
-                }
-                if (found) {
-                    // Since we now know the inputStream is ready, we can safely initialize the parser
-                    // We can't do that earlier since the parser will start pulling from the stream and if it is not ready,
-                    // The parser will automatically release its resources
-                    if (!parserInitialized) {
-                        parser.initialize(in, recType, true);
-                        parserInitialized = true;
-                    }
-                    tb.reset();
-                    if (propagateInput) {
-                        frameTuple.reset(tupleAccessor, tupleIndex);
-                        for (int i = 0; i < propagatedFields.length; i++) {
-                            dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                    frameTuple.getFieldStart(propagatedFields[i]),
-                                    frameTuple.getFieldLength(propagatedFields[i]));
-                            tb.addFieldEndOffset();
-                        }
-                    }
-                    parser.parse(tb.getDataOutput());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                } else if (propagateInput && retainNull) {
-                    tb.reset();
-                    frameTuple.reset(tupleAccessor, tupleIndex);
-                    for (int i = 0; i < propagatedFields.length; i++) {
-                        dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                frameTuple.getFieldStart(propagatedFields[i]),
-                                frameTuple.getFieldLength(propagatedFields[i]));
-                        tb.addFieldEndOffset();
-                    }
-                    dos.write(nullTupleBuild.getByteArray());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                }
-                tupleIndex++;
-            }
-        } catch (Exception e) {
-            // un expected error, we try to close the inputstream and throw an exception
-            try {
-                in.close();
-            } catch (IOException e1) {
-                e1.printStackTrace();
-            }
-            throw new HyracksDataException(e);
-        }
-    }
-
-    // For debugging
-    public void prettyPrint(FrameTupleAccessor tupleAccessor, RecordDescriptor recDesc) {
-        ByteBufferInputStream bbis = new ByteBufferInputStream();
-        DataInputStream dis = new DataInputStream(bbis);
-        int tc = tupleAccessor.getTupleCount();
-        System.err.println("TC: " + tc);
-        for (int i = 0; i < tc; ++i) {
-            System.err.print(
-                    i + ":(" + tupleAccessor.getTupleStartOffset(i) + ", " + tupleAccessor.getTupleEndOffset(i) + ")[");
-            for (int j = 0; j < tupleAccessor.getFieldCount(); ++j) {
-                System.err.print(j + ":(" + tupleAccessor.getFieldStartOffset(i, j) + ", "
-                        + tupleAccessor.getFieldEndOffset(i, j) + ") ");
-                System.err.print("{");
-                bbis.setByteBuffer(tupleAccessor.getBuffer(), tupleAccessor.getTupleStartOffset(i)
-                        + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(i, j));
-                try {
-                    byte tag = dis.readByte();
-                    if (tag == nullByte) {
-                        System.err.print("NULL");
-                    } else {
-                        bbis.setByteBuffer(tupleAccessor.getBuffer(), tupleAccessor.getTupleStartOffset(i)
-                                + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(i, j));
-                        System.err.print(recDesc.getFields()[j].deserialize(dis));
-                    }
-                } catch (IOException e) {
-                    e.printStackTrace();
-                }
-                System.err.print("}");
-            }
-            System.err.println("]");
-        }
-    }
-
-    protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
-            }
-        }
-
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
deleted file mode 100644
index 6abcbb8..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.InputStream;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.indexing.input.AbstractHDFSReader;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.operators.file.IDataParser;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-
-public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
-
-    private ArrayTupleBuilder tb;
-    private final FrameTupleAppender appender;
-    private final ARecordType recType;
-    private final IDataParser parser;
-    private final AMutableInt32 aMutableInt = new AMutableInt32(0);
-    private final AMutableInt64 aMutableLong = new AMutableInt64(0);
-
-    @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer intSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINT32);
-    @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer longSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINT64);
-
-    public AdmOrDelimitedIndexingTupleParser(IHyracksCommonContext ctx, ARecordType recType, IDataParser parser)
-            throws HyracksDataException {
-        this.parser = parser;
-        this.recType = recType;
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        tb = new ArrayTupleBuilder(3);
-    }
-
-    @Override
-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-        // Cast the input stream to a record reader
-        AbstractHDFSReader inReader = (AbstractHDFSReader) in;
-        try {
-            parser.initialize(in, recType, true);
-            while (true) {
-                tb.reset();
-                if (!parser.parse(tb.getDataOutput())) {
-                    break;
-                }
-                tb.addFieldEndOffset();
-                appendIndexingData(tb, inReader);
-                addTupleToFrame(writer);
-            }
-            appender.flush(writer, true);
-        } catch (AsterixException ae) {
-            throw new HyracksDataException(ae);
-        } catch (Exception ioe) {
-            throw new HyracksDataException(ioe);
-        }
-    }
-
-    // This function is used to append RID to Hyracks tuple
-    @SuppressWarnings("unchecked")
-    private void appendIndexingData(ArrayTupleBuilder tb, AbstractHDFSReader inReader) throws Exception {
-        aMutableInt.setValue(inReader.getFileNumber());
-        aMutableLong.setValue(inReader.getReaderPosition());
-        tb.addField(intSerde, aMutableInt);
-        tb.addField(longSerde, aMutableLong);
-    }
-
-    private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException("Record is too big to fit in a frame");
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
deleted file mode 100644
index b38b835..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.util.Map;
-
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.ADMDataParser;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.DelimitedDataParser;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * This is the parser factory for parsers used to do indexing
- */
-public class HDFSIndexingParserFactory implements ITupleParserFactory {
-
-    private static final long serialVersionUID = 1L;
-    // file input-format <text, seq, rc>
-    private final String inputFormat;
-    // content format <adm, delimited-text, binary>
-    private final String format;
-    // delimiter in case of delimited text
-    private final char delimiter;
-    // quote in case of delimited text
-    private final char quote;
-    // parser class name in case of binary format
-    private final String parserClassName;
-    // the expected data type
-    private final ARecordType atype;
-    // the hadoop job conf
-    private transient JobConf jobConf;
-    // adapter arguments
-    private Map<String, String> arguments;
-
-    public HDFSIndexingParserFactory(ARecordType atype, String inputFormat, String format, char delimiter, char quote,
-            String parserClassName) {
-        this.inputFormat = inputFormat;
-        this.format = format;
-        this.parserClassName = parserClassName;
-        this.delimiter = delimiter;
-        this.quote = quote;
-        this.atype = atype;
-    }
-
-    @Override
-    public ITupleParser createTupleParser(IHyracksCommonContext ctx) throws HyracksDataException {
-        if (format == null) {
-            throw new IllegalArgumentException("Unspecified data format");
-        }
-        if (inputFormat == null) {
-            throw new IllegalArgumentException("Unspecified data format");
-        }
-        if (!inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_RC)
-                && !inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_TEXT)
-                && !inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE)) {
-            throw new IllegalArgumentException("External Indexing not supportd for format " + inputFormat);
-        }
-        // Do some real work here
-        /*
-         * Choices are:
-         * 1. TxtOrSeq (Object) indexing tuple parser
-         * 2. RC indexing tuple parser
-         * 3. textual data tuple parser
-         */
-        if (format.equalsIgnoreCase(AsterixTupleParserFactory.FORMAT_ADM)) {
-            // choice 3 with adm data parser
-            ADMDataParser dataParser = new ADMDataParser();
-            return new AdmOrDelimitedIndexingTupleParser(ctx, atype, dataParser);
-        } else if (format.equalsIgnoreCase(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
-            // choice 3 with delimited data parser
-            DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDelimitedDataParser(atype, delimiter, quote);
-            return new AdmOrDelimitedIndexingTupleParser(ctx, atype, dataParser);
-        }
-
-        // binary data with a special parser --> create the parser
-        IAsterixHDFSRecordParser objectParser;
-        if (parserClassName.equalsIgnoreCase(HDFSAdapterFactory.PARSER_HIVE)) {
-            objectParser = new HiveObjectParser();
-        } else {
-            try {
-                objectParser = (IAsterixHDFSRecordParser) Class.forName(parserClassName).newInstance();
-            } catch (Exception e) {
-                throw new HyracksDataException("Unable to create object parser", e);
-            }
-        }
-        try {
-            objectParser.initialize(atype, arguments, jobConf);
-        } catch (Exception e) {
-            throw new HyracksDataException("Unable to initialize object parser", e);
-        }
-
-        if (inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
-            // Case 2
-            return new RCFileIndexingTupleParser(ctx, atype, objectParser);
-        } else {
-            // Case 1
-            return new TextOrSeqIndexingTupleParser(ctx, atype, objectParser);
-        }
-    }
-
-    public JobConf getJobConf() {
-        return jobConf;
-    }
-
-    public void setJobConf(JobConf jobConf) {
-        this.jobConf = jobConf;
-    }
-
-    public Map<String, String> getArguments() {
-        return arguments;
-    }
-
-    public void setArguments(Map<String, String> arguments) {
-        this.arguments = arguments;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
deleted file mode 100644
index d9ce7aa..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.util.Map;
-
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
-import org.apache.asterix.external.dataset.adapter.IControlledAdapter;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.input.RCFileLookupReader;
-import org.apache.asterix.external.indexing.input.SequenceFileLookupInputStream;
-import org.apache.asterix.external.indexing.input.SequenceFileLookupReader;
-import org.apache.asterix.external.indexing.input.TextFileLookupInputStream;
-import org.apache.asterix.external.indexing.input.TextFileLookupReader;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.ADMDataParser;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.DelimitedDataParser;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class HDFSLookupAdapter implements IControlledAdapter, Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    private RecordDescriptor inRecDesc;
-    private boolean propagateInput;
-    private int[] ridFields;
-    private int[] propagatedFields;
-    private IAType atype;
-    private Map<String, String> configuration;
-    private IHyracksTaskContext ctx;
-    private IControlledTupleParser parser;
-    private ExternalFileIndexAccessor fileIndexAccessor;
-    private boolean retainNull;
-
-    public HDFSLookupAdapter(IAType atype, RecordDescriptor inRecDesc, Map<String, String> adapterConfiguration,
-            boolean propagateInput, int[] ridFields, int[] propagatedFields, IHyracksTaskContext ctx,
-            ExternalFileIndexAccessor fileIndexAccessor, boolean retainNull) {
-        this.configuration = adapterConfiguration;
-        this.atype = atype;
-        this.ctx = ctx;
-        this.inRecDesc = inRecDesc;
-        this.propagatedFields = propagatedFields;
-        this.propagateInput = propagateInput;
-        this.propagatedFields = propagatedFields;
-        this.fileIndexAccessor = fileIndexAccessor;
-        this.ridFields = ridFields;
-        this.retainNull = retainNull;
-    }
-
-    /*
-     * This function is not easy to read and could be refactored into a better structure but for now it works
-     */
-    @Override
-    public void initialize(IHyracksTaskContext ctx, INullWriterFactory iNullWriterFactory) throws Exception {
-        JobConf jobConf = HDFSAdapterFactory.configureJobConf(configuration);
-        // Create the lookup reader and the controlled parser
-        if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
-            configureRCFile(jobConf, iNullWriterFactory);
-        } else if (configuration.get(AsterixTupleParserFactory.KEY_FORMAT)
-                .equals(AsterixTupleParserFactory.FORMAT_ADM)) {
-            // create an adm parser
-            ADMDataParser dataParser = new ADMDataParser();
-            if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
-                // Text input format
-                TextFileLookupInputStream in = new TextFileLookupInputStream(fileIndexAccessor, jobConf);
-                parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
-                        inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
-            } else {
-                // Sequence input format
-                SequenceFileLookupInputStream in = new SequenceFileLookupInputStream(fileIndexAccessor, jobConf);
-                parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
-                        inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
-            }
-        } else if (configuration.get(AsterixTupleParserFactory.KEY_FORMAT)
-                .equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
-            // create a delimited text parser
-            char delimiter = AsterixTupleParserFactory.getDelimiter(configuration);
-            char quote = AsterixTupleParserFactory.getQuote(configuration, delimiter);
-
-            DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDelimitedDataParser((ARecordType) atype,
-                    delimiter, quote);
-            if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
-                // Text input format
-                TextFileLookupInputStream in = new TextFileLookupInputStream(fileIndexAccessor, jobConf);
-                parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
-                        inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
-            } else {
-                // Sequence input format
-                SequenceFileLookupInputStream in = new SequenceFileLookupInputStream(fileIndexAccessor, jobConf);
-                parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
-                        inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
-            }
-        } else {
-            configureGenericSeqOrText(jobConf, iNullWriterFactory);
-        }
-    }
-
-    private void configureGenericSeqOrText(JobConf jobConf, INullWriterFactory iNullWriterFactory) throws IOException {
-        if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
-            // Text input format
-            TextFileLookupReader reader = new TextFileLookupReader(fileIndexAccessor, jobConf);
-            parser = new SeqOrTxtControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
-                    propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
-        } else {
-            // Sequence input format
-            SequenceFileLookupReader reader = new SequenceFileLookupReader(fileIndexAccessor, jobConf);
-            parser = new SeqOrTxtControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
-                    propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer, IFrameWriter writer) throws Exception {
-        parser.parseNext(writer, buffer);
-    }
-
-    @Override
-    public void close(IFrameWriter writer) throws Exception {
-        parser.close(writer);
-    }
-
-    @Override
-    public void fail() throws Exception {
-        // Do nothing
-    }
-
-    private void configureRCFile(Configuration jobConf, INullWriterFactory iNullWriterFactory)
-            throws IOException, Exception {
-        // RCFileLookupReader
-        RCFileLookupReader reader = new RCFileLookupReader(fileIndexAccessor,
-                HDFSAdapterFactory.configureJobConf(configuration));
-        parser = new RCFileControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
-                propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
-    }
-
-    private IAsterixHDFSRecordParser createRecordParser(Configuration jobConf) throws HyracksDataException {
-        // Create the record parser
-        // binary data with a special parser --> create the parser
-        IAsterixHDFSRecordParser objectParser;
-        if (configuration.get(HDFSAdapterFactory.KEY_PARSER).equals(HDFSAdapterFactory.PARSER_HIVE)) {
-            objectParser = new HiveObjectParser();
-        } else {
-            try {
-                objectParser = (IAsterixHDFSRecordParser) Class
-                        .forName(configuration.get(HDFSAdapterFactory.KEY_PARSER)).newInstance();
-            } catch (Exception e) {
-                throw new HyracksDataException("Unable to create object parser", e);
-            }
-        }
-        // initialize the parser
-        try {
-            objectParser.initialize((ARecordType) atype, configuration, jobConf);
-        } catch (Exception e) {
-            throw new HyracksDataException("Unable to initialize object parser", e);
-        }
-
-        return objectParser;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java
deleted file mode 100644
index fab507d..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.util.Map;
-
-import org.apache.asterix.external.adapter.factory.IControlledAdapterFactory;
-import org.apache.asterix.external.dataset.adapter.IControlledAdapter;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-
-// This class takes care of creating the adapter based on the formats and input format
-public class HDFSLookupAdapterFactory implements IControlledAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private Map<String, String> adapterConfiguration;
-    private IAType atype;
-    private boolean propagateInput;
-    private int[] ridFields;
-    private int[] propagatedFields;
-    private boolean retainNull;
-
-    @Override
-    public void configure(IAType atype, boolean propagateInput, int[] ridFields,
-            Map<String, String> adapterConfiguration, boolean retainNull) {
-        this.adapterConfiguration = adapterConfiguration;
-        this.atype = atype;
-        this.propagateInput = propagateInput;
-        this.ridFields = ridFields;
-        this.retainNull = retainNull;
-    }
-
-    @Override
-    public IControlledAdapter createAdapter(IHyracksTaskContext ctx, ExternalFileIndexAccessor fileIndexAccessor,
-            RecordDescriptor inRecDesc) {
-        if (propagateInput) {
-            configurePropagatedFields(inRecDesc);
-        }
-        return new HDFSLookupAdapter(atype, inRecDesc, adapterConfiguration, propagateInput, ridFields,
-                propagatedFields, ctx, fileIndexAccessor, retainNull);
-    }
-
-    private void configurePropagatedFields(RecordDescriptor inRecDesc) {
-        int ptr = 0;
-        boolean skip = false;
-        propagatedFields = new int[inRecDesc.getFieldCount() - ridFields.length];
-        for (int i = 0; i < inRecDesc.getFieldCount(); i++) {
-            if (ptr < ridFields.length) {
-                skip = false;
-                for (int j = 0; j < ridFields.length; j++) {
-                    if (ridFields[j] == i) {
-                        ptr++;
-                        skip = true;
-                        break;
-                    }
-                }
-                if (!skip)
-                    propagatedFields[i - ptr] = i;
-            } else {
-                propagatedFields[i - ptr] = i;
-            }
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
deleted file mode 100644
index f42a6d1..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.InputStream;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.indexing.input.AbstractHDFSReader;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-
-/*
- * This class is used with hdfs objects instead of hdfs
- */
-public class HDFSObjectTupleParser implements ITupleParser{
-
-    private ArrayTupleBuilder tb;
-    private final FrameTupleAppender appender;
-    private IAsterixHDFSRecordParser deserializer;
-
-    public HDFSObjectTupleParser(IHyracksCommonContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer)
-            throws HyracksDataException {
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        this.deserializer = deserializer;
-        tb = new ArrayTupleBuilder(1);
-    }
-
-    @Override
-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-        AbstractHDFSReader reader = (AbstractHDFSReader) in;
-        Object object;
-        try {
-            reader.initialize();
-            object = reader.readNext();
-            while (object!= null) {
-                tb.reset();
-                deserializer.parse(object, tb.getDataOutput());
-                tb.addFieldEndOffset();
-                addTupleToFrame(writer);
-                object = reader.readNext();
-            }
-            appender.flush(writer, true);
-        } catch (AsterixException ae) {
-            throw new HyracksDataException(ae);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
deleted file mode 100644
index ac3a92f..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.util.Map;
-
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public class HDFSObjectTupleParserFactory implements ITupleParserFactory{
-    private static final long serialVersionUID = 1L;
-    // parser class name in case of binary format
-    private String parserClassName;
-    // the expected data type
-    private ARecordType atype;
-    // the hadoop job conf
-    private HDFSAdapterFactory adapterFactory;
-    // adapter arguments
-    private Map<String,String> arguments;
-
-    public HDFSObjectTupleParserFactory(ARecordType atype, HDFSAdapterFactory adapterFactory, Map<String,String> arguments){
-        this.parserClassName = (String) arguments.get(HDFSAdapterFactory.KEY_PARSER);
-        this.atype = atype;
-        this.arguments = arguments;
-        this.adapterFactory = adapterFactory;
-    }
-
-    @Override
-    public ITupleParser createTupleParser(IHyracksCommonContext ctx) throws HyracksDataException {
-        IAsterixHDFSRecordParser objectParser;
-        if (parserClassName.equals(HDFSAdapterFactory.PARSER_HIVE)) {
-            objectParser = new HiveObjectParser();
-        } else {
-            try {
-                objectParser = (IAsterixHDFSRecordParser) Class.forName(parserClassName).newInstance();
-            } catch (Exception e) {
-                throw new HyracksDataException("Unable to create object parser", e);
-            }
-        }
-        try {
-            objectParser.initialize(atype, arguments, adapterFactory.getJobConf());
-        } catch (Exception e) {
-            throw new HyracksDataException("Unable to initialize object parser", e);
-        }
-
-        return new HDFSObjectTupleParser(ctx, atype, objectParser);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java
deleted file mode 100644
index ff5bc27..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.DataOutput;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.asterix.om.types.ARecordType;
-
-/**
- * This interface is provided for users to implements in order to support their own
- * it should be included sometimes in the future in the external library
- * input parsing
- * @author alamouda
- *
- */
-public interface IAsterixHDFSRecordParser {
-
-    /**
-     * This method is called once upon creating the serde before starting to parse objects
-     * @param record
-     *  The description of the expected dataset record.
-     * @param arguments
-     *  The arguments passed when creating the external dataset
-     */
-    public void initialize(ARecordType record, Map<String, String> arguments, Configuration hadoopConfig) throws Exception;
-    
-    /**
-     * This function takes an object, parse it and then serialize it into an adm record in the output buffer
-     * @param object
-     *  the serialized I/O object
-     * @param output
-     *  output buffer where deserialized object need to be serialized
-     */
-    public void parse(Object object, DataOutput output) throws Exception;
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParser.java
deleted file mode 100644
index c8fc6c2..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParser.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * This interface is to be implemented by parsers used in a pipelined hyracks job where input is not ready all at once
- */
-public interface IControlledTupleParser {
-    /**
-     * This function should flush the tuples setting in the frame writer buffer
-     * and free all resources
-     */
-    public void close(IFrameWriter writer) throws Exception;
-
-    /**
-     * This function is called when there are more data ready for parsing in the input stream
-     * @param writer
-     *          a frame writer that is used to push outgoig frames 
-     * @param frameBuffer 
-     *          a frame buffer containing the incoming tuples, used for propagating fields.
-     */
-    public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException;
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java
deleted file mode 100644
index 52d5123..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-public interface IControlledTupleParserFactory {
-    public IControlledTupleParser createTupleParser();
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
deleted file mode 100644
index c8e9c65..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.input.RCFileLookupReader;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.INullWriter;
-import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-
-public class RCFileControlledTupleParser implements IControlledTupleParser {
-
-    private ArrayTupleBuilder tb;
-    private transient DataOutput dos;
-    private final FrameTupleAppender appender;
-    private boolean propagateInput;
-    private int[] propagatedFields;
-    private FrameTupleReference frameTuple;
-    private IAsterixHDFSRecordParser parser;
-    private RCFileLookupReader reader;
-    private int[] ridFields;
-    private RecordDescriptor inRecDesc;
-    private FrameTupleAccessor tupleAccessor;
-    private ByteBufferInputStream bbis;
-    private DataInputStream dis;
-    private boolean retainNull;
-    protected byte nullByte;
-    protected ArrayTupleBuilder nullTupleBuild;
-
-    public RCFileControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser,
-            RCFileLookupReader reader, boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc,
-            int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        this.parser = parser;
-        this.reader = reader;
-        this.propagateInput = propagateInput;
-        this.propagatedFields = propagatedFields;
-        this.retainNull = retainNull;
-        this.inRecDesc = inRecDesc;
-        this.ridFields = ridFields;
-        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
-        if (propagateInput) {
-            tb = new ArrayTupleBuilder(propagatedFields.length + 1);
-        } else {
-            tb = new ArrayTupleBuilder(1);
-        }
-        frameTuple = new FrameTupleReference();
-        dos = tb.getDataOutput();
-        bbis = new ByteBufferInputStream();
-        dis = new DataInputStream(bbis);
-        nullByte = ATypeTag.NULL.serialize();
-        if (retainNull) {
-            INullWriter nullWriter = iNullWriterFactory.createNullWriter();
-            nullTupleBuild = new ArrayTupleBuilder(1);
-            DataOutput out = nullTupleBuild.getDataOutput();
-            try {
-                nullWriter.writeNull(out);
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        } else {
-            nullTupleBuild = null;
-        }
-    }
-
-    @Override
-    public void close(IFrameWriter writer) throws Exception {
-        try {
-            reader.close();
-            appender.flush(writer, true);
-        } catch (IOException ioe) {
-            throw new HyracksDataException(ioe);
-        }
-    }
-
-    @Override
-    public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
-        try {
-            int tupleCount = 0;
-            int tupleIndex = 0;
-            Object object;
-            tupleAccessor.reset(frameBuffer);
-            tupleCount = tupleAccessor.getTupleCount();
-            int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
-            // Loop over tuples
-            while (tupleIndex < tupleCount) {
-                int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
-                int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
-                        ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
-                // Check if null <- for outer join ->
-                if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
-                    object = null;
-                } else {
-                    // Get file number
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
-                    int fileNumber = ((AInt32) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]].deserialize(dis))
-                                    .getIntegerValue();
-                    // Get record group offset
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + tupleAccessor.getFieldStartOffset(tupleIndex,
-                            ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
-                    long recordOffset = ((AInt64) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]].deserialize(dis))
-                                    .getLongValue();
-                    // Get row number
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + tupleAccessor.getFieldStartOffset(tupleIndex,
-                            ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]));
-                    int rowNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]]
-                            .deserialize(dis)).getIntegerValue();
-
-                    // Read record from external source
-                    object = reader.read(fileNumber, recordOffset, rowNumber);
-                }
-                if (object != null) {
-                    tb.reset();
-                    if (propagateInput) {
-                        frameTuple.reset(tupleAccessor, tupleIndex);
-                        for (int i = 0; i < propagatedFields.length; i++) {
-                            dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                    frameTuple.getFieldStart(propagatedFields[i]),
-                                    frameTuple.getFieldLength(propagatedFields[i]));
-                            tb.addFieldEndOffset();
-                        }
-                    }
-                    // parse record
-                    parser.parse(object, tb.getDataOutput());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                } else if (propagateInput && retainNull) {
-                    tb.reset();
-                    frameTuple.reset(tupleAccessor, tupleIndex);
-                    for (int i = 0; i < propagatedFields.length; i++) {
-                        dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                frameTuple.getFieldStart(propagatedFields[i]),
-                                frameTuple.getFieldLength(propagatedFields[i]));
-                        tb.addFieldEndOffset();
-                    }
-                    dos.write(nullTupleBuild.getByteArray());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                }
-                tupleIndex++;
-            }
-        } catch (Exception e) {
-            // Something went wrong, try to close the reader and then throw an exception <-this should never happen->
-            try {
-                reader.close();
-            } catch (Exception e1) {
-                e.addSuppressed(e1);
-            }
-            throw new HyracksDataException(e);
-        }
-    }
-
-    protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
deleted file mode 100644
index eaa3381..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import org.apache.asterix.external.indexing.input.AbstractHDFSReader;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public class RCFileIndexingTupleParser extends AbstractIndexingTupleParser{
-
-    private Integer rowNumber = -1;
-    private Integer lastFileNumber = -1;
-    private long lastByteLocation = -1;
-
-    public RCFileIndexingTupleParser(IHyracksCommonContext ctx, ARecordType recType, IAsterixHDFSRecordParser
-            deserializer)
-            throws HyracksDataException {
-        super(ctx, recType, deserializer);
-        tb = new ArrayTupleBuilder(4);
-        dos = tb.getDataOutput();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    protected void appendIndexingData(ArrayTupleBuilder tb,
-            AbstractHDFSReader inReader) throws Exception {
-        aMutableInt.setValue(inReader.getFileNumber());
-        aMutableLong.setValue(inReader.getReaderPosition());
-        //add file number
-        tb.addField(intSerde, aMutableInt);
-        //add record offset
-        tb.addField(longSerde, aMutableLong);
-        //add row number
-        if(aMutableInt.getIntegerValue().equals(lastFileNumber) && aMutableLong.getLongValue() == lastByteLocation){
-            rowNumber++;
-        }else{
-            lastFileNumber = aMutableInt.getIntegerValue();
-            lastByteLocation = aMutableLong.getLongValue();
-            rowNumber = 0;
-        }
-        aMutableInt.setValue(rowNumber);
-        tb.addField(intSerde, aMutableInt);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
deleted file mode 100644
index 23ddd8a..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.input.ILookupReader;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.INullWriter;
-import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-
-public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
-
-    private ArrayTupleBuilder tb;
-    private transient DataOutput dos;
-    private final FrameTupleAppender appender;
-    private boolean propagateInput;
-    private int[] propagatedFields;
-    private FrameTupleReference frameTuple;
-    private IAsterixHDFSRecordParser parser;
-    private ILookupReader reader;
-    private int[] ridFields;
-    private RecordDescriptor inRecDesc;
-    private FrameTupleAccessor tupleAccessor;
-    private ByteBufferInputStream bbis;
-    private DataInputStream dis;
-    private boolean retainNull;
-    protected byte nullByte;
-    protected ArrayTupleBuilder nullTupleBuild;
-
-    public SeqOrTxtControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser, ILookupReader reader,
-            boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc, int[] ridFields,
-            boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
-        appender = new FrameTupleAppender(new VSizeFrame(ctx));
-        this.parser = parser;
-        this.reader = reader;
-        this.propagateInput = propagateInput;
-        this.ridFields = ridFields;
-        this.retainNull = retainNull;
-        if (propagateInput) {
-            tb = new ArrayTupleBuilder(propagatedFields.length + 1);
-            frameTuple = new FrameTupleReference();
-            this.propagatedFields = propagatedFields;
-        } else {
-            tb = new ArrayTupleBuilder(1);
-        }
-        dos = tb.getDataOutput();
-        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
-        bbis = new ByteBufferInputStream();
-        dis = new DataInputStream(bbis);
-        nullByte = ATypeTag.NULL.serialize();
-        if (retainNull) {
-            INullWriter nullWriter = iNullWriterFactory.createNullWriter();
-            nullTupleBuild = new ArrayTupleBuilder(1);
-            DataOutput out = nullTupleBuild.getDataOutput();
-            try {
-                nullWriter.writeNull(out);
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        } else {
-            nullTupleBuild = null;
-        }
-    }
-
-    @Override
-    public void close(IFrameWriter writer) throws Exception {
-        try {
-            reader.close();
-            appender.flush(writer, true);
-        } catch (IOException ioe) {
-            throw new HyracksDataException(ioe);
-        }
-    }
-
-    @Override
-    public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
-        try {
-            int tupleCount = 0;
-            int tupleIndex = 0;
-            Object record;
-            tupleAccessor.reset(frameBuffer);
-            tupleCount = tupleAccessor.getTupleCount();
-            int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
-            // Loop over incoming tuples
-            while (tupleIndex < tupleCount) {
-                int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
-                int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
-                        ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
-                // Check if null <- for outer join ->
-                if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
-                    record = null;
-                } else {
-                    // Get file number
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
-                    int fileNumber = ((AInt32) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]].deserialize(dis))
-                                    .getIntegerValue();
-                    // Get record offset
-                    bbis.setByteBuffer(frameBuffer, tupleStartOffset + tupleAccessor.getFieldStartOffset(tupleIndex,
-                            ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
-                    long recordOffset = ((AInt64) inRecDesc
-                            .getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]].deserialize(dis))
-                                    .getLongValue();
-                    // Read the record
-                    record = reader.read(fileNumber, recordOffset);
-                }
-                if (record != null) {
-                    tb.reset();
-                    if (propagateInput) {
-                        frameTuple.reset(tupleAccessor, tupleIndex);
-                        for (int i = 0; i < propagatedFields.length; i++) {
-                            dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                    frameTuple.getFieldStart(propagatedFields[i]),
-                                    frameTuple.getFieldLength(propagatedFields[i]));
-                            tb.addFieldEndOffset();
-                        }
-                    }
-                    // parse it
-                    parser.parse(record, tb.getDataOutput());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                } else if (propagateInput && retainNull) {
-                    tb.reset();
-                    frameTuple.reset(tupleAccessor, tupleIndex);
-                    for (int i = 0; i < propagatedFields.length; i++) {
-                        dos.write(frameTuple.getFieldData(propagatedFields[i]),
-                                frameTuple.getFieldStart(propagatedFields[i]),
-                                frameTuple.getFieldLength(propagatedFields[i]));
-                        tb.addFieldEndOffset();
-                    }
-                    dos.write(nullTupleBuild.getByteArray());
-                    tb.addFieldEndOffset();
-                    addTupleToFrame(writer);
-                }
-                tupleIndex++;
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            try {
-                reader.close();
-            } catch (Exception e2) {
-                e.addSuppressed(e2);
-            }
-            throw new HyracksDataException(e);
-        }
-    }
-
-    private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
deleted file mode 100644
index d44b3f3..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.dataflow;
-
-import org.apache.asterix.external.indexing.input.AbstractHDFSReader;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public class TextOrSeqIndexingTupleParser extends AbstractIndexingTupleParser{
-    public TextOrSeqIndexingTupleParser(IHyracksCommonContext ctx,
-            ARecordType recType, IAsterixHDFSRecordParser deserializer)
-            throws HyracksDataException {
-        super(ctx, recType, deserializer);
-        tb = new ArrayTupleBuilder(3);
-        dos = tb.getDataOutput();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    protected void appendIndexingData(ArrayTupleBuilder tb,
-            AbstractHDFSReader inReader) throws Exception {
-        aMutableInt.setValue(inReader.getFileNumber());
-        aMutableLong.setValue(inReader.getReaderPosition());
-
-        tb.addField(intSerde, aMutableInt);
-        tb.addField(longSerde, aMutableLong);
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java
deleted file mode 100644
index 563a46d..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-
-/*
- * This class is used for seek and read of external data of format adm or delimited text in sequence of text input format
- */
-public abstract class AbstractHDFSLookupInputStream extends InputStream {
-
-    protected String pendingValue = null;
-    protected FileSystem fs;
-    protected int fileNumber = -1;
-    protected int EOL = "\n".getBytes()[0];
-    protected boolean skipFile = false;
-    protected ExternalFile file = new ExternalFile(null, null, 0, null, null, 0, ExternalFilePendingOp.PENDING_NO_OP);
-    protected ExternalFileIndexAccessor filesIndexAccessor;
-
-    public AbstractHDFSLookupInputStream(ExternalFileIndexAccessor filesIndexAccessor, JobConf conf)
-            throws IOException {
-        this.filesIndexAccessor = filesIndexAccessor;
-        fs = FileSystem.get(conf);
-    }
-
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        if (pendingValue != null) {
-            int size = pendingValue.length() + 1;
-            if (size > len) {
-                return 0;
-            }
-            System.arraycopy(pendingValue.getBytes(), 0, buffer, offset, pendingValue.length());
-            buffer[offset + pendingValue.length()] = (byte) EOL;
-            pendingValue = null;
-            return size;
-        }
-        return -1;
-    }
-
-    public boolean fetchRecord(int fileNumber, long recordOffset) throws Exception {
-        if (fileNumber != this.fileNumber) {
-            // New file number
-            this.fileNumber = fileNumber;
-            filesIndexAccessor.searchForFile(fileNumber, file);
-
-            try {
-                FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
-                if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
-                    this.fileNumber = fileNumber;
-                    skipFile = true;
-                    return false;
-                } else {
-                    this.fileNumber = fileNumber;
-                    skipFile = false;
-                    openFile(file.getFileName());
-                }
-            } catch (FileNotFoundException e) {
-                // We ignore File not found exceptions <- it means file was deleted and so we don't care about it anymore ->
-                this.fileNumber = fileNumber;
-                skipFile = true;
-                return false;
-            }
-        } else if (skipFile) {
-            return false;
-        }
-        return read(recordOffset);
-    }
-
-    @Override
-    public int read() throws IOException {
-        return 0;
-    }
-
-    protected abstract boolean read(long byteLocation);
-
-    protected abstract void openFile(String fileName) throws IOException;
-
-    @Override
-    public void close() throws IOException {
-        super.close();
-    }
-
-    public ExternalFileIndexAccessor getExternalFileIndexAccessor() {
-        return filesIndexAccessor;
-    }
-
-    public void setExternalFileIndexAccessor(ExternalFileIndexAccessor filesIndexAccessor) {
-        this.filesIndexAccessor = filesIndexAccessor;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSReader.java
deleted file mode 100644
index 65bfcf3..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/AbstractHDFSReader.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.InputStream;
-
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.Reporter;
-
-/***
- * an abstract class to be used for reading hdfs based datasets one record at a time <- used for indexing->
- */
-public abstract class AbstractHDFSReader extends InputStream {
-
-    /***
-     * This function should be called once to do initial setup before starting to read records
-     *
-     * @return true if ready for reading
-     */
-    abstract public boolean initialize() throws Exception;
-
-    /***
-     * @return the next object read or null if reached end of stream
-     */
-    abstract public Object readNext() throws Exception;
-
-    /**
-     * @return the file name of the current filesplit being read
-     * @throws Exception
-     *             in case of end of records is reached
-     */
-    abstract public String getFileName() throws Exception;
-
-    /**
-     * @return return the reader position of last record read
-     * @throws Exception
-     *             in case of end of records is reached
-     */
-    abstract public long getReaderPosition() throws Exception;
-
-    /**
-     * @return the file number of the file being read
-     * @throws Exception
-     */
-    abstract public int getFileNumber() throws Exception;
-
-    protected Reporter getReporter() {
-        Reporter reporter = new Reporter() {
-
-            @Override
-            public Counter getCounter(Enum<?> arg0) {
-                return null;
-            }
-
-            @Override
-            public Counter getCounter(String arg0, String arg1) {
-                return null;
-            }
-
-            @Override
-            public InputSplit getInputSplit() throws UnsupportedOperationException {
-                return null;
-            }
-
-            @Override
-            public void incrCounter(Enum<?> arg0, long arg1) {
-            }
-
-            @Override
-            public void incrCounter(String arg0, String arg1, long arg2) {
-            }
-
-            @Override
-            public void setStatus(String arg0) {
-            }
-
-            @Override
-            public void progress() {
-            }
-
-            @Override
-            public float getProgress() {
-                return 0.0f;
-            }
-        };
-
-        return reporter;
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericFileAwareRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericFileAwareRecordReader.java
deleted file mode 100644
index ba36407..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericFileAwareRecordReader.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-
-/**
- * This is a generic reader used for indexing external dataset or for performing full scan for external dataset with
- * a stored snapshot
- *
- * @author alamouda
- */
-
-public class GenericFileAwareRecordReader extends GenericRecordReader {
-
-    private List<ExternalFile> files;
-    private FileSystem hadoopFS;
-    private long recordOffset = 0L;
-
-    public GenericFileAwareRecordReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
-            boolean[] executed, List<ExternalFile> files) throws IOException {
-        super(inputSplits, readSchedule, nodeName, conf, executed);
-        this.files = files;
-        hadoopFS = FileSystem.get(conf);
-    }
-
-    private boolean moveToNext() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (executed) {
-                    if (executed[currentSplitIndex] == false) {
-                        executed[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-
-                /**
-                 * read the split
-                 */
-                try {
-                    String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
-                    FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
-                    //skip if not the same file stored in the files snapshot
-                    if (fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime()
-                            .getTime())
-                        continue;
-                    reader = getRecordReader(currentSplitIndex);
-                } catch (Exception e) {
-                    continue;
-                }
-                key = reader.createKey();
-                value = reader.createValue();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public Object readNext() throws IOException {
-
-        if (reader == null) {
-            return null;
-        }
-        recordOffset = reader.getPos();
-        if (reader.next(key, value)) {
-            return value;
-        }
-        while (moveToNext()) {
-            recordOffset = reader.getPos();
-            if (reader.next(key, value)) {
-                return value;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public String getFileName() throws Exception {
-        return files.get(currentSplitIndex).getFileName();
-    }
-
-    @Override
-    public long getReaderPosition() throws Exception {
-        return recordOffset;
-    }
-
-    @Override
-    public int getFileNumber() throws Exception {
-        return files.get(currentSplitIndex).getFileNumber();
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericRecordReader.java
deleted file mode 100644
index ab050a7..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/GenericRecordReader.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-
-/**
- * This class can be used by any input format to perform full scan operations
- */
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class GenericRecordReader extends AbstractHDFSReader {
-
-    protected RecordReader reader;
-    protected Object key;
-    protected Object value;
-    protected int currentSplitIndex = 0;
-    protected boolean executed[];
-    protected InputSplit[] inputSplits;
-    protected String[] readSchedule;
-    protected String nodeName;
-    protected JobConf conf;
-
-    public GenericRecordReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
-            boolean executed[]) {
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-        this.executed = executed;
-    }
-
-    @Override
-    public boolean initialize() throws IOException {
-        return moveToNext();
-    }
-
-    private boolean moveToNext() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (executed) {
-                    if (executed[currentSplitIndex] == false) {
-                        executed[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-
-                /**
-                 * read the split
-                 */
-                reader = getRecordReader(currentSplitIndex);
-                key = reader.createKey();
-                value = reader.createValue();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    protected RecordReader getRecordReader(int slitIndex) throws IOException {
-        RecordReader reader = conf.getInputFormat().getRecordReader(inputSplits[slitIndex], conf, getReporter());
-        return reader;
-    }
-
-    @Override
-    public Object readNext() throws IOException {
-        if (reader == null) {
-            return null;
-        }
-        if (reader.next(key, value)) {
-            return value;
-        }
-        while (moveToNext()) {
-            if (reader.next(key, value)) {
-                return value;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public String getFileName() throws Exception {
-        return null;
-    }
-
-    @Override
-    public long getReaderPosition() throws Exception {
-        return reader.getPos();
-    }
-
-    @Override
-    public int getFileNumber() throws Exception {
-        throw new NotImplementedException("This reader doesn't support this function");
-    }
-
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        throw new NotImplementedException("Use readNext()");
-    }
-
-    @Override
-    public int read() throws IOException {
-        throw new NotImplementedException("Use readNext()");
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/ILookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/ILookupReader.java
deleted file mode 100644
index d48aaf7..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/ILookupReader.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-public interface ILookupReader {
-    public Object read(int fileNumber, long recordOffset) throws Exception;
-    public void close();
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileDataReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileDataReader.java
deleted file mode 100644
index 50853d4..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileDataReader.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-
-//Used in two cases:
-//1. building an index over a dataset
-//2. performing full scan over a dataset that has built index (to provide consistent view) with RCFile format
-
-@SuppressWarnings("rawtypes")
-public class RCFileDataReader extends AbstractHDFSReader {
-
-    private RecordReader reader;
-    private Object key;
-    private Object value;
-    private int currentSplitIndex = 0;
-    private String fileName;
-    private long recordGroupOffset;
-    private long nextRecordGroupOffset;
-    private boolean executed[];
-    private InputSplit[] inputSplits;
-    private String[] readSchedule;
-    private String nodeName;
-    private JobConf conf;
-    private List<ExternalFile> files;
-    private FileSystem hadoopFS;
-
-    public RCFileDataReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
-            boolean executed[], List<ExternalFile> files) throws IOException {
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-        this.files = files;
-        hadoopFS = FileSystem.get(conf);
-    }
-
-    private boolean moveToNext() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (executed) {
-                    if (executed[currentSplitIndex] == false) {
-                        executed[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-
-                /**
-                 * read the split
-                 */
-                try {
-                    if (files != null) {
-                        fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
-                        FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
-                        //skip if not the same file stored in the files snapshot
-                        if (fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime()
-                                .getTime())
-                            continue;
-                    }
-                    reader = getRecordReader(currentSplitIndex);
-                    recordGroupOffset = -1;
-                    nextRecordGroupOffset = reader.getPos();
-                } catch (Exception e) {
-                    continue;
-                }
-                key = reader.createKey();
-                value = reader.createValue();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        throw new NotImplementedException("Use readNext()");
-    }
-
-    @Override
-    public int read() throws IOException {
-        throw new NotImplementedException("Use readNext()");
-    }
-
-    private RecordReader getRecordReader(int slitIndex) throws IOException {
-        RecordReader reader;
-        try {
-            reader = conf.getInputFormat().getRecordReader(inputSplits[slitIndex], conf, getReporter());
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-        return reader;
-    }
-
-    @Override
-    public boolean initialize() throws IOException {
-        return moveToNext();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public Object readNext() throws IOException {
-
-        if (reader == null) {
-            return null;
-        }
-        if (reader.next(key, value)) {
-            if (reader.getPos() != nextRecordGroupOffset) {
-                recordGroupOffset = nextRecordGroupOffset;
-                nextRecordGroupOffset = reader.getPos();
-            }
-            return value;
-        }
-        while (moveToNext()) {
-            if (reader.next(key, value)) {
-                if (reader.getPos() != nextRecordGroupOffset) {
-                    recordGroupOffset = nextRecordGroupOffset;
-                    nextRecordGroupOffset = reader.getPos();
-                }
-                return value;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public String getFileName() throws Exception {
-        return files.get(currentSplitIndex).getFileName();
-    }
-
-    @Override
-    public long getReaderPosition() throws Exception {
-        return recordGroupOffset;
-    }
-
-    @Override
-    public int getFileNumber() throws Exception {
-        return files.get(currentSplitIndex).getFileNumber();
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileLookupReader.java
deleted file mode 100644
index f312228..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/RCFileLookupReader.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-
-public class RCFileLookupReader {
-    private FileSystem fs;
-    private Configuration conf;
-    private int fileNumber = -1;
-    private int rowNumber;
-    private long recordGroupOffset;
-    private Reader reader;
-    boolean skipFile = false;
-    private LongWritable rcKey = new LongWritable();
-    private BytesRefArrayWritable rcValue = new BytesRefArrayWritable();
-    private ExternalFile currentFile = new ExternalFile(null, null, 0, null, null, 0L,
-            ExternalFilePendingOp.PENDING_NO_OP);
-    private ExternalFileIndexAccessor filesIndexAccessor;
-
-    public RCFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf) throws IOException {
-        fs = FileSystem.get(conf);
-        this.conf = conf;
-        this.filesIndexAccessor = filesIndexAccessor;
-    }
-
-    public Writable read(int fileNumber, long recordGroupOffset, int rowNumber) throws Exception {
-        if (fileNumber != this.fileNumber) {
-            filesIndexAccessor.searchForFile(fileNumber, currentFile);
-            try {
-                FileStatus fileStatus = fs.getFileStatus(new Path(currentFile.getFileName()));
-                if (fileStatus.getModificationTime() != currentFile.getLastModefiedTime().getTime()) {
-                    this.fileNumber = fileNumber;
-                    skipFile = true;
-                    return null;
-                } else {
-                    this.fileNumber = fileNumber;
-                    skipFile = false;
-                }
-            } catch (FileNotFoundException e) {
-                // Couldn't find file, skip it
-                this.fileNumber = fileNumber;
-                skipFile = true;
-                return null;
-            }
-            // Close old file and open new one
-            if (reader != null)
-                reader.close();
-            reader = new Reader(fs, new Path(currentFile.getFileName()), conf);
-            this.recordGroupOffset = -1;
-            this.rowNumber = -1;
-        } else if (skipFile) {
-            return null;
-        }
-        // Seek to the record group if needed
-        if (recordGroupOffset != this.recordGroupOffset) {
-            this.recordGroupOffset = recordGroupOffset;
-            if (reader.getPosition() != recordGroupOffset)
-                reader.seek(recordGroupOffset);
-            reader.resetBuffer();
-            this.rowNumber = -1;
-        }
-
-        // skip rows to the record row
-        while (this.rowNumber < rowNumber) {
-            reader.next(rcKey);
-            reader.getCurrentRow(rcValue);
-            this.rowNumber++;
-        }
-        return rcValue;
-    }
-
-    public void close() throws Exception {
-        if (reader != null)
-            reader.close();
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupInputStream.java
deleted file mode 100644
index e787921..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupInputStream.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.util.ReflectionUtils;
-
-@SuppressWarnings("deprecation")
-public class SequenceFileLookupInputStream extends AbstractHDFSLookupInputStream {
-
-    private SequenceFile.Reader reader;
-    private Writable seqKey;
-    private Text seqValue = new Text();
-    private Configuration conf;
-
-    public SequenceFileLookupInputStream(ExternalFileIndexAccessor fileIndexAccessor, JobConf conf) throws IOException {
-        super(fileIndexAccessor, conf);
-        this.conf = conf;
-    }
-
-    @Override
-    protected void openFile(String fileName) throws IOException {
-        if (reader != null) {
-            reader.close();
-        }
-        reader = new SequenceFile.Reader(fs, new Path(fileName), conf);
-        seqKey = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (reader != null) {
-            reader.close();
-        }
-        super.close();
-    }
-
-    @Override
-    protected boolean read(long recordOffset) {
-        try {
-            reader.seek(recordOffset);
-            reader.next(seqKey, seqValue);
-            pendingValue = seqValue.toString();
-            return true;
-        } catch (IOException e) {
-            // Same Question: seek and read failed afer openning file succeede, should we do something about it?
-            e.printStackTrace();
-            return false;
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupReader.java
deleted file mode 100644
index 76b3660..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/SequenceFileLookupReader.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-
-public class SequenceFileLookupReader implements ILookupReader {
-
-    private Reader reader;
-    private Writable key;
-    private Writable value;
-    private FileSystem fs;
-    private int fileNumber = -1;
-    private boolean skipFile = false;
-    private ExternalFile file = new ExternalFile(null, null, 0, null, null, 0L, ExternalFilePendingOp.PENDING_NO_OP);
-    private ExternalFileIndexAccessor filesIndexAccessor;
-    private Configuration conf;
-
-    public SequenceFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf)
-            throws IOException {
-        fs = FileSystem.get(conf);
-        this.filesIndexAccessor = filesIndexAccessor;
-        this.conf = conf;
-    }
-
-    @Override
-    public Writable read(int fileNumber, long recordOffset) throws Exception {
-        if (fileNumber != this.fileNumber) {
-            //get file name
-            this.fileNumber = fileNumber;
-            filesIndexAccessor.searchForFile(fileNumber, file);
-            try {
-                FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
-                if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
-                    this.fileNumber = fileNumber;
-                    skipFile = true;
-                    return null;
-                } else {
-                    this.fileNumber = fileNumber;
-                    skipFile = false;
-                    openFile(file.getFileName());
-                }
-            } catch (FileNotFoundException e) {
-                // file was not found, do nothing and skip its tuples
-                this.fileNumber = fileNumber;
-                skipFile = true;
-                return null;
-            }
-        } else if (skipFile) {
-            return null;
-        }
-        reader.seek(recordOffset);
-        reader.next(key, value);
-        return value;
-    }
-
-    @SuppressWarnings("deprecation")
-    private void openFile(String FileName) throws IOException {
-        if (reader != null)
-            try {
-                reader.close();
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        reader = new SequenceFile.Reader(fs, new Path(FileName), conf);
-        key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-        value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-    }
-
-    @Override
-    public void close() {
-        if (reader != null)
-            try {
-                reader.close();
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupInputStream.java
deleted file mode 100644
index ea82c18..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupInputStream.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-
-public class TextFileLookupInputStream extends AbstractHDFSLookupInputStream {
-
-    private HDFSSeekableLineReader lineReader = new HDFSSeekableLineReader();
-    private Text value = new Text();
-
-    public TextFileLookupInputStream(ExternalFileIndexAccessor filesIndexAccessor, JobConf conf) throws IOException {
-        super(filesIndexAccessor, conf);
-    }
-
-    @Override
-    public void openFile(String FileName) throws IOException {
-        if (lineReader.getReader() != null) {
-            lineReader.getReader().close();
-        }
-        lineReader.resetReader(fs.open(new Path(FileName)));
-    }
-
-    @Override
-    public void close() {
-        if (lineReader.getReader() != null) {
-            try {
-                lineReader.getReader().close();
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    @Override
-    protected boolean read(long recordOffset) {
-        try {
-            lineReader.seek(recordOffset);
-            lineReader.readLine(value);
-            pendingValue = value.toString();
-            return true;
-        } catch (IOException e) {
-            // file was opened and then when trying to seek and read, an error occurred <- should we throw an exception ???->
-            e.printStackTrace();
-            return false;
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupReader.java
deleted file mode 100644
index 5864df2..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextFileLookupReader.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-
-public class TextFileLookupReader implements ILookupReader {
-    private FileSystem fs;
-    private int fileNumber = -1;
-    private boolean skipFile = false;
-    private ExternalFile file = new ExternalFile(null, null, 0, null, null, 0L, ExternalFilePendingOp.PENDING_NO_OP);
-    private ExternalFileIndexAccessor filesIndexAccessor;
-    private HDFSSeekableLineReader lineReader;
-    private Text value = new Text();
-
-    public TextFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf) throws IOException {
-        this.fs = FileSystem.get(conf);
-        this.filesIndexAccessor = filesIndexAccessor;
-        this.lineReader = new HDFSSeekableLineReader();
-    }
-
-    @Override
-    public String read(int fileNumber, long recordOffset) throws Exception {
-        if (fileNumber != this.fileNumber) {
-            this.fileNumber = fileNumber;
-            filesIndexAccessor.searchForFile(fileNumber, file);
-
-            try {
-                FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
-                if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
-                    this.fileNumber = fileNumber;
-                    skipFile = true;
-                    return null;
-                } else {
-                    this.fileNumber = fileNumber;
-                    skipFile = false;
-                    openFile(file.getFileName());
-                }
-            } catch (FileNotFoundException e) {
-                // File is not there, skip it and do nothing
-                this.fileNumber = fileNumber;
-                skipFile = true;
-                return null;
-            }
-        } else if (skipFile) {
-            return null;
-        }
-        lineReader.seek(recordOffset);
-        lineReader.readLine(value);
-        return value.toString();
-    }
-
-    private void openFile(String FileName) throws IOException {
-        if (lineReader.getReader() != null) {
-            lineReader.getReader().close();
-        }
-        lineReader.resetReader(fs.open(new Path(FileName)));
-    }
-
-    @Override
-    public void close() {
-        if (lineReader.getReader() != null) {
-            try {
-                lineReader.getReader().close();
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualDataReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualDataReader.java
deleted file mode 100644
index 5e4f013..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualDataReader.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-
-// Used in two cases:
-// 1. building an index over a dataset
-// 2. performing full scan over a dataset that has built index (to provide consistent view)
-
-@SuppressWarnings("rawtypes")
-public class TextualDataReader extends AbstractHDFSReader {
-
-    private RecordReader<Object, Text> reader;
-    private Object key;
-    private Text value;
-    private boolean hasMore = false;
-    private int EOL = "\n".getBytes()[0];
-    private Text pendingValue = null;
-    private int currentSplitIndex = 0;
-    private String fileName;
-    private long recordOffset;
-    private boolean executed[];
-    private InputSplit[] inputSplits;
-    private String[] readSchedule;
-    private String nodeName;
-    private JobConf conf;
-    private List<ExternalFile> files;
-    private FileSystem hadoopFS;
-
-    public TextualDataReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
-            boolean executed[], List<ExternalFile> files) throws IOException {
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-        this.files = files;
-        hadoopFS = FileSystem.get(conf);
-    }
-
-    @Override
-    public boolean initialize() throws Exception {
-        return moveToNext();
-    }
-
-    @Override
-    public Object readNext() throws Exception {
-        if (reader == null) {
-            return null;
-        }
-        recordOffset = reader.getPos();
-        if (reader.next(key, value)) {
-            return value;
-        }
-        while (moveToNext()) {
-            recordOffset = reader.getPos();
-            if (reader.next(key, value)) {
-                return value;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public int getFileNumber() throws Exception {
-        return files.get(currentSplitIndex).getFileNumber();
-    }
-
-    @Override
-    public String getFileName() throws Exception {
-        return files.get(currentSplitIndex).getFileName();
-    }
-
-    @Override
-    public long getReaderPosition() throws Exception {
-        return recordOffset;
-    }
-
-    @Override
-    public int read() throws IOException {
-        throw new NotImplementedException("Use read(byte[], int, int");
-    }
-
-    @SuppressWarnings("unchecked")
-    private boolean moveToNext() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (executed) {
-                    if (executed[currentSplitIndex] == false) {
-                        executed[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-
-                /**
-                 * read the split
-                 */
-                try {
-                    if (files != null) {
-                        fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
-                        FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
-                        // Skip if not the same file stored in the files snapshot
-                        if (fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime()
-                                .getTime())
-                            continue;
-                    }
-                    // It is the same file
-                    reader = getRecordReader(currentSplitIndex);
-                } catch (Exception e) {
-                    // ignore exceptions <-- This might change later -->
-                    continue;
-                }
-                key = reader.createKey();
-                value = reader.createValue();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    private RecordReader getRecordReader(int splitIndex) throws IOException {
-        RecordReader reader;
-        if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-            SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-            reader = format.getRecordReader(inputSplits[splitIndex], conf, getReporter());
-        } else {
-            TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-            reader = format.getRecordReader(inputSplits[splitIndex], conf, getReporter());
-        }
-        return reader;
-    }
-
-    // Return one record at a time <to preserve the indexing information>
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        if (reader == null) {
-            if (!moveToNext()) {
-                // nothing to read
-                return -1;
-            }
-        }
-
-        int numBytes = 0;
-        if (pendingValue != null) {
-            int sizeOfNextTuple = pendingValue.getLength() + 1;
-            if (sizeOfNextTuple > len) {
-                return 0;
-            }
-            System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-            buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-            numBytes += pendingValue.getLength() + 1;
-            pendingValue = null;
-            return numBytes;
-        }
-        if (numBytes < len) {
-            //store the byte location
-            recordOffset = reader.getPos();
-            hasMore = reader.next(key, value);
-            if (!hasMore) {
-                while (moveToNext()) {
-                    //store the byte location
-                    recordOffset = reader.getPos();
-                    hasMore = reader.next(key, value);
-                    if (hasMore) {
-                        //return the value read
-                        int sizeOfNextTuple = value.getLength() + 1;
-                        if (numBytes + sizeOfNextTuple > len) {
-                            pendingValue = value;
-                            return 0;
-                        } else {
-                            System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-                            buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-                            numBytes += sizeOfNextTuple;
-                            return numBytes;
-                        }
-                    }
-                }
-                return -1;
-            } else {
-                //return the value read
-                int sizeOfNextTuple = value.getLength() + 1;
-                if (numBytes + sizeOfNextTuple > len) {
-                    pendingValue = value;
-                    return 0;
-                } else {
-                    System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-                    buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-                    numBytes += sizeOfNextTuple;
-                    return numBytes;
-                }
-            }
-        }
-        return numBytes;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualFullScanDataReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualFullScanDataReader.java
deleted file mode 100644
index 9fe09a2..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/TextualFullScanDataReader.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.indexing.input;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.Counters.Counter;
-
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-
-public class TextualFullScanDataReader extends InputStream {
-
-    private RecordReader<Object, Text> reader;
-    private Object key;
-    private Text value;
-    private boolean hasMore = false;
-    private int EOL = "\n".getBytes()[0];
-    private Text pendingValue = null;
-    private int currentSplitIndex = 0;
-    private boolean executed[];
-    private InputSplit[] inputSplits;
-    private String[] readSchedule;
-    private String nodeName;
-    private JobConf conf;
-
-    public TextualFullScanDataReader(boolean executed[], InputSplit[] inputSplits, String[] readSchedule,
-            String nodeName, JobConf conf) {
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-    }
-
-    @Override
-    public int available() {
-        return 1;
-    }
-
-    @SuppressWarnings("unchecked")
-    private boolean moveToNext() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (executed) {
-                    if (executed[currentSplitIndex] == false) {
-                        executed[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-
-                /**
-                 * read the split
-                 */
-                reader = getRecordReader(currentSplitIndex);
-                key = reader.createKey();
-                value = (Text) reader.createValue();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        if (reader == null) {
-            if (!moveToNext()) {
-                // nothing to read
-                return -1;
-            }
-        }
-
-        int numBytes = 0;
-        if (pendingValue != null) {
-            int sizeOfNextTuple = pendingValue.getLength() + 1;
-            if (sizeOfNextTuple > len) {
-                return 0;
-            }
-            System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-            buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-            numBytes += pendingValue.getLength() + 1;
-            pendingValue = null;
-        }
-
-        while (numBytes < len) {
-            hasMore = reader.next(key, value);
-            if (!hasMore) {
-                while (moveToNext()) {
-                    hasMore = reader.next(key, value);
-                    if (hasMore) {
-                        // move to the next non-empty split
-                        break;
-                    }
-                }
-            }
-            if (!hasMore) {
-                return (numBytes == 0) ? -1 : numBytes;
-            }
-            int sizeOfNextTuple = value.getLength() + 1;
-            if (numBytes + sizeOfNextTuple > len) {
-                // cannot add tuple to current buffer
-                // but the reader has moved pass the fetched tuple
-                // we need to store this for a subsequent read call.
-                // and return this then.
-                pendingValue = value;
-                break;
-            } else {
-                System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-                buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-                numBytes += sizeOfNextTuple;
-            }
-        }
-        return numBytes;
-    }
-
-    @Override
-    public int read() throws IOException {
-        throw new NotImplementedException("Use read(byte[], int, int");
-    }
-
-    @SuppressWarnings("rawtypes")
-    private RecordReader getRecordReader(int splitIndex) throws IOException {
-        if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-            SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-            RecordReader reader = format.getRecordReader((org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex],
-                    conf, getReporter());
-            return reader;
-        } else {
-            TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-            RecordReader reader = format.getRecordReader((org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex],
-                    conf, getReporter());
-            return reader;
-        }
-    }
-
-    private Reporter getReporter() {
-        Reporter reporter = new Reporter() {
-
-            @Override
-            public Counter getCounter(Enum<?> arg0) {
-                return null;
-            }
-
-            @Override
-            public Counter getCounter(String arg0, String arg1) {
-                return null;
-            }
-
-            @Override
-            public InputSplit getInputSplit() throws UnsupportedOperationException {
-                return null;
-            }
-
-            @Override
-            public void incrCounter(Enum<?> arg0, long arg1) {
-            }
-
-            @Override
-            public void incrCounter(String arg0, String arg1, long arg2) {
-            }
-
-            @Override
-            public void setStatus(String arg0) {
-            }
-
-            @Override
-            public void progress() {
-            }
-
-            @Override
-            public float getProgress() {
-                return 0.0f;
-            }
-        };
-
-        return reporter;
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
new file mode 100644
index 0000000..7e9fdcb
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IIndexibleExternalDataSource;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.IndexingScheduler;
+import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.external.input.stream.HDFSInputStreamProvider;
+import org.apache.asterix.external.provider.ExternalIndexerProvider;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.hdfs.dataflow.ConfFactory;
+import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
+import org.apache.hyracks.hdfs.scheduler.Scheduler;
+
+public class HDFSDataSourceFactory
+        implements IInputStreamProviderFactory, IRecordReaderFactory<Object>, IIndexibleExternalDataSource {
+
+    protected static final long serialVersionUID = 1L;
+    protected transient AlgebricksPartitionConstraint clusterLocations;
+    protected String[] readSchedule;
+    protected boolean read[];
+    protected InputSplitsFactory inputSplitsFactory;
+    protected ConfFactory confFactory;
+    protected boolean configured = false;
+    protected static Scheduler hdfsScheduler;
+    protected static IndexingScheduler indexingScheduler;
+    protected static Boolean initialized = false;
+    protected List<ExternalFile> files;
+    protected Map<String, String> configuration;
+    protected Class<?> recordClass;
+    protected boolean indexingOp = false;
+    private JobConf conf;
+    private InputSplit[] inputSplits;
+    private String nodeName;
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        if (!HDFSDataSourceFactory.initialized) {
+            HDFSDataSourceFactory.initialize();
+        }
+        this.configuration = configuration;
+        JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
+        confFactory = new ConfFactory(conf);
+        clusterLocations = getPartitionConstraint();
+        int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+        // if files list was set, we restrict the splits to the list
+        InputSplit[] inputSplits;
+        if (files == null) {
+            inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+        } else {
+            inputSplits = HDFSUtils.getSplits(conf, files);
+        }
+        if (indexingOp) {
+            readSchedule = indexingScheduler.getLocationConstraints(inputSplits);
+        } else {
+            readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
+        }
+        inputSplitsFactory = new InputSplitsFactory(inputSplits);
+        read = new boolean[readSchedule.length];
+        Arrays.fill(read, false);
+        if (!ExternalDataUtils.isDataSourceStreamProvider(configuration)) {
+            RecordReader<?, ?> reader = conf.getInputFormat().getRecordReader(inputSplits[0], conf, Reporter.NULL);
+            this.recordClass = reader.createValue().getClass();
+            reader.close();
+        }
+    }
+
+    // Used to tell the factory to restrict the splits to the intersection between this list and the actual files on hdfs side
+    @Override
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) {
+        this.files = files;
+        this.indexingOp = indexingOp;
+    }
+
+    /*
+     * The method below was modified to take care of the following
+     * 1. when target files are not null, it generates a file aware input stream that validate against the files
+     * 2. if the data is binary, it returns a generic reader
+     */
+    @Override
+    public IInputStreamProvider createInputStreamProvider(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        try {
+            if (!configured) {
+                conf = confFactory.getConf();
+                inputSplits = inputSplitsFactory.getSplits();
+                nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+                configured = true;
+            }
+            return new HDFSInputStreamProvider<Object>(read, inputSplits, readSchedule, nodeName, conf, configuration,
+                    files);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * Get the cluster locations for this input stream factory. This method specifies on which asterix nodes the
+     * external
+     * adapter will run and how many threads per node.
+     * @return
+     */
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() {
+        clusterLocations = HDFSUtils.getPartitionConstraints(clusterLocations);
+        return clusterLocations;
+    }
+
+    /**
+     * This method initialize the scheduler which assigns responsibility of reading different logical input splits from
+     * HDFS
+     */
+    private static void initialize() {
+        synchronized (initialized) {
+            if (!initialized) {
+                hdfsScheduler = HDFSUtils.initializeHDFSScheduler();
+                indexingScheduler = HDFSUtils.initializeIndexingHDFSScheduler();
+                initialized = true;
+            }
+        }
+    }
+
+    public JobConf getJobConf() throws HyracksDataException {
+        return confFactory.getConf();
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return (ExternalDataUtils.isDataSourceStreamProvider(configuration)) ? DataSourceType.STREAM
+                : DataSourceType.RECORDS;
+    }
+
+    @Override
+    public IRecordReader<? extends Writable> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws Exception {
+        JobConf conf = confFactory.getConf();
+        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+        HDFSRecordReader<Object, Writable> recordReader = new HDFSRecordReader<Object, Writable>(read, inputSplits,
+                readSchedule, nodeName, conf);
+        if (files != null) {
+            recordReader.setSnapshot(files);
+            recordReader.setIndexer(ExternalIndexerProvider.getIndexer(configuration));
+        }
+        recordReader.configure(configuration);
+        return recordReader;
+    }
+
+    @Override
+    public Class<?> getRecordClass() {
+        return recordClass;
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return true;
+    }
+
+    @Override
+    public boolean isIndexingOp() {
+        return (files != null && indexingOp);
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
new file mode 100644
index 0000000..fd5c397
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record;
+
+import java.util.Arrays;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.util.ExternalDataConstants;
+
+public class CharArrayRecord implements IRawRecord<char[]> {
+
+    private char[] value;
+    private int size;
+
+    @Override
+    public byte[] getBytes() {
+        return new String(value).getBytes();
+    }
+
+    @Override
+    public char[] get() {
+        return value;
+    }
+
+    @Override
+    public int size() {
+        return size;
+    }
+
+    public CharArrayRecord(int initialCapacity) {
+        value = new char[initialCapacity];
+        size = 0;
+    }
+
+    public CharArrayRecord() {
+        value = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+        size = 0;
+    }
+
+    public void setValue(char[] recordBuffer, int offset, int length) {
+        if (value.length < length) {
+            value = new char[length];
+        }
+        System.arraycopy(recordBuffer, offset, value, 0, length);
+        size = length;
+    }
+
+    private void ensureCapacity(int len) {
+        if (value.length < len) {
+            value = Arrays.copyOf(value, (int) (len * 1.25));
+        }
+    }
+
+    public void append(char[] recordBuffer, int offset, int length) {
+        ensureCapacity(size + length);
+        System.arraycopy(recordBuffer, offset, value, size, length);
+        size += length;
+    }
+
+    @Override
+    public void reset() {
+        size = 0;
+    }
+
+    @Override
+    public String toString() {
+        return String.valueOf(value, 0, size);
+    }
+
+    public void setValue(char[] value) {
+        this.value = value;
+    }
+
+    public void endRecord() {
+        if (value[size - 1] != ExternalDataConstants.LF) {
+            appendChar(ExternalDataConstants.LF);
+        }
+    }
+
+    private void appendChar(char c) {
+        ensureCapacity(size + 1);
+        value[size] = c;
+        size++;
+    }
+
+    @Override
+    public Class<char[]> getRecordClass() {
+        return char[].class;
+    }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
new file mode 100644
index 0000000..365bc22
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record;
+
+import org.apache.asterix.external.api.IRawRecord;
+
+public class GenericRecord<T> implements IRawRecord<T> {
+
+    private T record;
+
+    public GenericRecord() {
+    }
+
+    public GenericRecord(T record) {
+        this.record = record;
+    }
+
+    @Override
+    public byte[] getBytes() {
+        return null;
+    }
+
+    @Override
+    public T get() {
+        return record;
+    }
+
+    @Override
+    public int size() {
+        return -1;
+    }
+
+    @Override
+    public Class<?> getRecordClass() {
+        return record.getClass();
+    }
+
+    public void set(T record) {
+        this.record = record;
+    }
+
+    @Override
+    public void reset() {
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java
new file mode 100644
index 0000000..1b84e7a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+
+public abstract class AbstractCharRecordLookupReader extends AbstractHDFSLookupRecordReader<char[]> {
+    public AbstractCharRecordLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
+            Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    protected CharArrayRecord record = new CharArrayRecord();
+    protected Text value = new Text();
+    protected CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder();
+    protected ByteBuffer reusableByteBuffer = ByteBuffer.allocateDirect(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+    protected CharBuffer reusableCharBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+
+    @Override
+    public Class<?> getRecordClass() throws IOException {
+        return char[].class;
+    }
+
+    @Override
+    protected IRawRecord<char[]> lookup(RecordId rid) throws IOException {
+        record.reset();
+        readRecord(rid);
+        writeRecord();
+        return record;
+    }
+
+    protected abstract void readRecord(RecordId rid) throws IOException;
+
+    private void writeRecord() {
+        reusableByteBuffer.clear();
+        if (reusableByteBuffer.remaining() < value.getLength()) {
+            reusableByteBuffer = ByteBuffer
+                    .allocateDirect(value.getLength() + ExternalDataConstants.DEFAULT_BUFFER_INCREMENT);
+        }
+        reusableByteBuffer.put(value.getBytes(), 0, value.getLength());
+        reusableByteBuffer.flip();
+        while (reusableByteBuffer.hasRemaining()) {
+            decoder.decode(reusableByteBuffer, reusableCharBuffer, false);
+            record.append(reusableCharBuffer.array(), 0, reusableCharBuffer.position());
+            reusableCharBuffer.clear();
+        }
+        record.endRecord();
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java
new file mode 100644
index 0000000..5a20962
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractHDFSLookupRecordReader<T> implements ILookupRecordReader<T> {
+
+    protected int fileId;
+    private ExternalFileIndexAccessor snapshotAccessor;
+    protected ExternalFile file;
+    protected FileSystem fs;
+    protected Configuration conf;
+    protected boolean replaced;
+
+    public AbstractHDFSLookupRecordReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
+            Configuration conf) {
+        this.snapshotAccessor = snapshotAccessor;
+        this.fs = fs;
+        this.conf = conf;
+        this.fileId = -1;
+        this.file = new ExternalFile();
+    }
+
+    @Override
+    public void configure(Map<String, String> configurations) throws Exception {
+    }
+
+    @Override
+    public IRawRecord<T> read(RecordId rid) throws Exception {
+        if (rid.getFileId() != fileId) {
+            // close current file
+            closeFile();
+            // lookup new file
+            snapshotAccessor.lookup(rid.getFileId(), file);
+            fileId = rid.getFileId();
+            try {
+                validate();
+                if (!replaced) {
+                    openFile();
+                    validate();
+                    if (replaced) {
+                        closeFile();
+                    }
+                }
+            } catch (FileNotFoundException e) {
+                replaced = true;
+            }
+        }
+        if (replaced) {
+            return null;
+        }
+        return lookup(rid);
+    }
+
+    protected abstract IRawRecord<T> lookup(RecordId rid) throws IOException;
+
+    private void validate() throws IllegalArgumentException, IOException {
+        FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
+        replaced = fileStatus.getModificationTime() != file.getLastModefiedTime().getTime();
+    }
+
+    protected abstract void closeFile();
+
+    protected abstract void openFile() throws IllegalArgumentException, IOException;
+
+    @Override
+    public final void open() throws HyracksDataException {
+        snapshotAccessor.open();
+    }
+
+    @Override
+    public void close() throws IOException {
+        try {
+            closeFile();
+        } finally {
+            snapshotAccessor.close();
+        }
+    }
+
+    @Override
+    public void fail() {
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java
new file mode 100644
index 0000000..3b59b98
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.input.stream.AInputStreamReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+
+public abstract class AbstractStreamRecordReader implements IRecordReader<char[]>, IIndexingDatasource {
+    protected AInputStreamReader reader;
+    protected CharArrayRecord record;
+    protected char[] inputBuffer;
+    protected int bufferLength = 0;
+    protected int bufferPosn = 0;
+    protected IExternalIndexer indexer;
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException {
+        return record;
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    public void setInputStream(AInputStream inputStream) throws IOException {
+        this.reader = new AInputStreamReader(inputStream);
+    }
+
+    @Override
+    public Class<char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new CharArrayRecord();
+        inputBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+    }
+
+    @Override
+    public IExternalIndexer getIndexer() {
+        return indexer;
+    }
+
+    @Override
+    public void setIndexer(IExternalIndexer indexer) {
+        this.indexer = indexer;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java
new file mode 100644
index 0000000..c7acb1a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexibleExternalDataSource;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public abstract class AbstractStreamRecordReaderFactory<T>
+        implements IRecordReaderFactory<T>, IIndexibleExternalDataSource {
+
+    private static final long serialVersionUID = 1L;
+    protected IInputStreamProviderFactory inputStreamFactory;
+    protected Map<String, String> configuration;
+
+    public AbstractStreamRecordReaderFactory<T> setInputStreamFactoryProvider(
+            IInputStreamProviderFactory inputStreamFactory) {
+        this.inputStreamFactory = inputStreamFactory;
+        return this;
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return inputStreamFactory.getPartitionConstraint();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        inputStreamFactory.configure(configuration);
+        configureStreamReaderFactory(configuration);
+    }
+
+    protected abstract void configureStreamReaderFactory(Map<String, String> configuration) throws Exception;
+
+    @Override
+    public boolean isIndexible() {
+        return inputStreamFactory.isIndexible();
+    }
+
+    @Override
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) throws Exception {
+        ((IIndexibleExternalDataSource) inputStreamFactory).setSnapshot(files, indexingOp);
+    }
+
+    @Override
+    public boolean isIndexingOp() {
+        if (inputStreamFactory.isIndexible()) {
+            return ((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp();
+        }
+        return false;
+    }
+
+    protected IRecordReader<char[]> configureReader(AbstractStreamRecordReader recordReader, IHyracksTaskContext ctx,
+            int partition) throws Exception {
+        IInputStreamProvider inputStreamProvider = inputStreamFactory.createInputStreamProvider(ctx, partition);
+        IExternalIndexer indexer = null;
+        if (inputStreamFactory.isIndexible()) {
+            if (((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp()) {
+                indexer = ((IIndexingDatasource) inputStreamProvider).getIndexer();
+            }
+        }
+        recordReader.setInputStream(inputStreamProvider.getInputStream());
+        recordReader.setIndexer(indexer);
+        recordReader.configure(configuration);
+        return recordReader;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/EmptyRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/EmptyRecordReader.java
new file mode 100644
index 0000000..e742b1e
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/EmptyRecordReader.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.RecordReader;
+
+public class EmptyRecordReader<K, V> implements RecordReader<K, V> {
+
+    @Override
+    public boolean next(K key, V value) throws IOException {
+        return false;
+    }
+
+    @Override
+    public K createKey() {
+        return null;
+    }
+
+    @Override
+    public V createValue() {
+        return null;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+        return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+        return 0;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java
new file mode 100644
index 0000000..d88f967
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class HDFSRecordReader<K, V extends Writable> implements IRecordReader<Writable>, IIndexingDatasource {
+
+    protected RecordReader<K, Writable> reader;
+    protected V value = null;
+    protected K key = null;
+    protected int currentSplitIndex = 0;
+    protected boolean read[];
+    protected InputFormat<?, ?> inputFormat;
+    protected InputSplit[] inputSplits;
+    protected String[] readSchedule;
+    protected String nodeName;
+    protected JobConf conf;
+    protected GenericRecord<Writable> record;
+    // Indexing variables
+    protected IExternalIndexer indexer;
+    protected List<ExternalFile> snapshot;
+    protected FileSystem hdfs;
+
+    public HDFSRecordReader(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf) {
+        this.read = read;
+        this.inputSplits = inputSplits;
+        this.readSchedule = readSchedule;
+        this.nodeName = nodeName;
+        this.conf = conf;
+        this.inputFormat = conf.getInputFormat();
+        this.reader = new EmptyRecordReader<K, Writable>();
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new GenericRecord<Writable>();
+        nextInputSplit();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        if (reader.next(key, value)) {
+            return true;
+        }
+        while (nextInputSplit()) {
+            if (reader.next(key, value)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public IRawRecord<Writable> next() throws IOException {
+        record.set(value);
+        return record;
+    }
+
+    @Override
+    public Class<? extends Writable> getRecordClass() throws IOException {
+        if (value == null) {
+            if (!nextInputSplit()) {
+                return null;
+            }
+        }
+        return value.getClass();
+    }
+
+    private boolean nextInputSplit() throws IOException {
+        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+            /**
+             * read all the partitions scheduled to the current node
+             */
+            if (readSchedule[currentSplitIndex].equals(nodeName)) {
+                /**
+                 * pick an unread split to read synchronize among
+                 * simultaneous partitions in the same machine
+                 */
+                synchronized (read) {
+                    if (read[currentSplitIndex] == false) {
+                        read[currentSplitIndex] = true;
+                    } else {
+                        continue;
+                    }
+                }
+                if (snapshot != null) {
+                    String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
+                    FileStatus fileStatus = hdfs.getFileStatus(new Path(fileName));
+                    // Skip if not the same file stored in the files snapshot
+                    if (fileStatus.getModificationTime() != snapshot.get(currentSplitIndex).getLastModefiedTime()
+                            .getTime())
+                        continue;
+                }
+
+                reader.close();
+                reader = getRecordReader(currentSplitIndex);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    private RecordReader<K, Writable> getRecordReader(int splitIndex) throws IOException {
+        reader = (RecordReader<K, Writable>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+        if (key == null) {
+            key = reader.createKey();
+            value = (V) reader.createValue();
+        }
+        if (indexer != null) {
+            try {
+                indexer.reset(this);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        return reader;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    @Override
+    public IExternalIndexer getIndexer() {
+        return indexer;
+    }
+
+    @Override
+    public void setIndexer(IExternalIndexer indexer) {
+        this.indexer = indexer;
+    }
+
+    public List<ExternalFile> getSnapshot() {
+        return snapshot;
+    }
+
+    public void setSnapshot(List<ExternalFile> snapshot) throws IOException {
+        this.snapshot = snapshot;
+        hdfs = FileSystem.get(conf);
+    }
+
+    public int getCurrentSplitIndex() {
+        return currentSplitIndex;
+    }
+
+    public RecordReader<K, Writable> getReader() {
+        return reader;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/HDFSSeekableLineReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
similarity index 93%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/HDFSSeekableLineReader.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
index 7916a16..9466a96 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/input/HDFSSeekableLineReader.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
@@ -16,17 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.input;
+package org.apache.asterix.external.input.record.reader;
 
 import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.io.Text;
 
-/**
- * A class that provides a line reader from an input stream which also allows performing seek operations
- */
-public class HDFSSeekableLineReader {
+public class HDFSTextLineReader {
     private static final int DEFAULT_BUFFER_SIZE = 32 * 1024;
     private int bufferSize = DEFAULT_BUFFER_SIZE;
     private FSDataInputStream reader;
@@ -52,7 +50,7 @@
      *            The input stream
      * @throws IOException
      */
-    public HDFSSeekableLineReader(FSDataInputStream in) throws IOException {
+    public HDFSTextLineReader(FSDataInputStream in) throws IOException {
         this(in, DEFAULT_BUFFER_SIZE);
     }
 
@@ -66,14 +64,14 @@
      *            Size of the read buffer
      * @throws IOException
      */
-    public HDFSSeekableLineReader(FSDataInputStream in, int bufferSize) throws IOException {
+    public HDFSTextLineReader(FSDataInputStream in, int bufferSize) throws IOException {
         this.reader = in;
         this.bufferSize = bufferSize;
         this.buffer = new byte[this.bufferSize];
         currentFilePos = in.getPos();
     }
 
-    public HDFSSeekableLineReader() throws IOException {
+    public HDFSTextLineReader() throws IOException {
         this.bufferSize = DEFAULT_BUFFER_SIZE;
         this.buffer = new byte[this.bufferSize];
     }
@@ -87,7 +85,7 @@
      *            configuration
      * @throws IOException
      */
-    public HDFSSeekableLineReader(FSDataInputStream in, Configuration conf) throws IOException {
+    public HDFSTextLineReader(FSDataInputStream in, Configuration conf) throws IOException {
         this(in, conf.getInt(KEY_BUFFER_SIZE, DEFAULT_BUFFER_SIZE));
     }
 
@@ -170,7 +168,7 @@
             }
         } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
 
-        if (bytesConsumed > (long) Integer.MAX_VALUE)
+        if (bytesConsumed > Integer.MAX_VALUE)
             throw new IOException("Too many bytes before newline: " + bytesConsumed);
         currentFilePos = reader.getPos() - bufferLength + bufferPosn;
         return (int) bytesConsumed;
@@ -229,4 +227,8 @@
         bufferPosn = 0;
         currentFilePos = reader.getPos();
     }
+
+    public void close() throws IOException {
+        reader.close();
+    }
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java
new file mode 100644
index 0000000..9b11df6
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+
+public class LineRecordReader extends AbstractStreamRecordReader {
+
+    protected boolean prevCharCR;
+    protected int newlineLength;
+    protected int recordNumber = 0;
+
+    @Override
+    public boolean hasNext() throws IOException {
+        /* We're reading data from in, but the head of the stream may be
+         * already buffered in buffer, so we have several cases:
+         * 1. No newline characters are in the buffer, so we need to copy
+         *    everything and read another buffer from the stream.
+         * 2. An unambiguously terminated line is in buffer, so we just
+         *    copy to record.
+         * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+         *    in CR.  In this case we copy everything up to CR to record, but
+         *    we also need to see what follows CR: if it's LF, then we
+         *    need consume LF as well, so next call to readLine will read
+         *    from after that.
+         * We use a flag prevCharCR to signal if previous character was CR
+         * and, if it happens to be at the end of the buffer, delay
+         * consuming it until we have a chance to look at the char that
+         * follows.
+         */
+        newlineLength = 0; //length of terminating newline
+        prevCharCR = false; //true of prev char was CR
+        record.reset();
+        int readLength = 0;
+        do {
+            int startPosn = bufferPosn; //starting from where we left off the last time
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    if (readLength > 0) {
+                        record.endRecord();
+                        recordNumber++;
+                        return true;
+                    }
+                    reader.close();
+                    return false; //EOF
+                }
+            }
+            for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+                if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
+                    newlineLength = (prevCharCR) ? 2 : 1;
+                    ++bufferPosn; // at next invocation proceed from following byte
+                    break;
+                }
+                if (prevCharCR) { //CR + notLF, we are at notLF
+                    newlineLength = 1;
+                    break;
+                }
+                prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
+            }
+            readLength = bufferPosn - startPosn;
+            if (prevCharCR && newlineLength == 0) {
+                --readLength; //CR at the end of the buffer
+            }
+            if (readLength > 0) {
+                record.append(inputBuffer, startPosn, readLength);
+            }
+        } while (newlineLength == 0);
+        recordNumber++;
+        return true;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        if (ExternalDataUtils.hasHeader(configuration)) {
+            if (hasNext()) {
+                next();
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java
new file mode 100644
index 0000000..3a82a68
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.ILookupReaderFactory;
+import org.apache.asterix.external.input.record.reader.factory.HDFSLookupReaderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+
+public class LookupReaderFactoryProvider {
+
+    @SuppressWarnings("rawtypes")
+    public static ILookupReaderFactory getLookupReaderFactory(Map<String, String> configuration) throws Exception {
+        String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
+        if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
+                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)
+                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
+            HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<Object>();
+            readerFactory.configure(configuration);
+            return readerFactory;
+        } else {
+            throw new AsterixException("Unrecognized external format");
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java
new file mode 100644
index 0000000..668876e
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataExceptionUtils;
+
+public class QuotedLineRecordReader extends LineRecordReader {
+
+    private char quote;
+    private boolean prevCharEscape;
+    private boolean inQuote;
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        if (quoteString == null || quoteString.length() != 1) {
+            throw new AsterixException(ExternalDataExceptionUtils.incorrectParameterMessage(
+                    ExternalDataConstants.KEY_QUOTE, ExternalDataConstants.PARAMETER_OF_SIZE_ONE, quoteString));
+        }
+        this.quote = quoteString.charAt(0);
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+        newlineLength = 0;
+        prevCharCR = false;
+        prevCharEscape = false;
+        record.reset();
+        int readLength = 0;
+        inQuote = false;
+        do {
+            int startPosn = bufferPosn;
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    {
+                        if (readLength > 0) {
+                            if (inQuote) {
+                                throw new IOException("malformed input record ended inside quote");
+                            }
+                            record.endRecord();
+                            recordNumber++;
+                            return true;
+                        }
+                        return false;
+                    }
+                }
+            }
+            for (; bufferPosn < bufferLength; ++bufferPosn) {
+                if (!inQuote) {
+                    if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
+                        newlineLength = (prevCharCR) ? 2 : 1;
+                        ++bufferPosn;
+                        break;
+                    }
+                    if (prevCharCR) {
+                        newlineLength = 1;
+                        break;
+                    }
+                    prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
+                    if (inputBuffer[bufferPosn] == quote) {
+                        if (!prevCharEscape) {
+                            inQuote = true;
+                        }
+                    }
+                    if (prevCharEscape) {
+                        prevCharEscape = false;
+                    } else {
+                        prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                    }
+                } else {
+                    // only look for next quote
+                    if (inputBuffer[bufferPosn] == quote) {
+                        if (!prevCharEscape) {
+                            inQuote = false;
+                        }
+                    }
+                    prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                }
+            }
+            readLength = bufferPosn - startPosn;
+            if (prevCharCR && newlineLength == 0) {
+                --readLength;
+            }
+            if (readLength > 0) {
+                record.append(inputBuffer, startPosn, readLength);
+            }
+        } while (newlineLength == 0);
+        recordNumber++;
+        return true;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java
new file mode 100644
index 0000000..5c33502
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+public class RCLookupReader extends AbstractHDFSLookupRecordReader<BytesRefArrayWritable> {
+    public RCLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(RCLookupReader.class.getName());
+    private Reader reader;
+    private LongWritable key = new LongWritable();
+    private BytesRefArrayWritable value = new BytesRefArrayWritable();
+    private GenericRecord<BytesRefArrayWritable> record = new GenericRecord<BytesRefArrayWritable>();
+    private long offset;
+    private int row;
+
+    @Override
+    public Class<?> getRecordClass() throws IOException {
+        return Writable.class;
+    }
+
+    @Override
+    protected IRawRecord<BytesRefArrayWritable> lookup(RecordId rid) throws IOException {
+        if (rid.getOffset() != offset) {
+            offset = rid.getOffset();
+            if (reader.getPosition() != offset)
+                reader.seek(offset);
+            reader.resetBuffer();
+            row = -1;
+        }
+
+        // skip rows to the record row
+        while (row < rid.getRow()) {
+            reader.next(key);
+            reader.getCurrentRow(value);
+            row++;
+        }
+        record.set(value);
+        return record;
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file", e);
+        }
+    }
+
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        reader = new Reader(fs, new Path(file.getFileName()), conf);
+        offset = -1;
+        row = -1;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java
new file mode 100644
index 0000000..1c2dc30
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.log4j.Logger;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+import com.sun.syndication.feed.synd.SyndFeed;
+import com.sun.syndication.fetcher.FeedFetcher;
+import com.sun.syndication.fetcher.FetcherEvent;
+import com.sun.syndication.fetcher.FetcherException;
+import com.sun.syndication.fetcher.FetcherListener;
+import com.sun.syndication.fetcher.impl.FeedFetcherCache;
+import com.sun.syndication.fetcher.impl.HashMapFeedInfoCache;
+import com.sun.syndication.fetcher.impl.HttpURLFeedFetcher;
+import com.sun.syndication.io.FeedException;
+
+public class RSSRecordReader implements IRecordReader<SyndEntryImpl> {
+
+    private static final Logger LOGGER = Logger.getLogger(RSSRecordReader.class.getName());
+    private boolean modified = false;
+    private Queue<SyndEntryImpl> rssFeedBuffer = new LinkedList<SyndEntryImpl>();
+    private FeedFetcherCache feedInfoCache;
+    private FeedFetcher fetcher;
+    private FetcherEventListenerImpl listener;
+    private URL feedUrl;
+    private GenericRecord<SyndEntryImpl> record = new GenericRecord<SyndEntryImpl>();
+    private boolean done = false;
+
+    public RSSRecordReader(String url) throws MalformedURLException {
+        feedUrl = new URL(url);
+        feedInfoCache = HashMapFeedInfoCache.getInstance();
+        fetcher = new HttpURLFeedFetcher(feedInfoCache);
+        listener = new FetcherEventListenerImpl(this, LOGGER);
+        fetcher.addFetcherEventListener(listener);
+    }
+
+    public boolean isModified() {
+        return modified;
+    }
+
+    @Override
+    public void close() throws IOException {
+        fetcher.removeFetcherEventListener(listener);
+    }
+
+    @Override
+    public void configure(Map<String, String> configurations) throws Exception {
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return !done;
+    }
+
+    @Override
+    public IRawRecord<SyndEntryImpl> next() throws IOException {
+        if (done) {
+            return null;
+        }
+        try {
+            SyndEntryImpl feedEntry;
+            feedEntry = getNextRSSFeed();
+            if (feedEntry == null) {
+                return null;
+            }
+            record.set(feedEntry);
+            return record;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public Class<SyndEntryImpl> getRecordClass() throws IOException {
+        return SyndEntryImpl.class;
+    }
+
+    @Override
+    public boolean stop() {
+        done = true;
+        return true;
+    }
+
+    public void setModified(boolean modified) {
+        this.modified = modified;
+    }
+
+    private SyndEntryImpl getNextRSSFeed() throws Exception {
+        if (rssFeedBuffer.isEmpty()) {
+            fetchFeed();
+        }
+        if (rssFeedBuffer.isEmpty()) {
+            return null;
+        } else {
+            return rssFeedBuffer.remove();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void fetchFeed() throws IllegalArgumentException, IOException, FeedException, FetcherException {
+        // Retrieve the feed.
+        // We will get a Feed Polled Event and then a
+        // Feed Retrieved event (assuming the feed is valid)
+        SyndFeed feed = fetcher.retrieveFeed(feedUrl);
+        if (modified) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info(feedUrl + " retrieved");
+                LOGGER.info(feedUrl + " has a title: " + feed.getTitle() + " and contains " + feed.getEntries().size()
+                        + " entries.");
+            }
+            List<? extends SyndEntryImpl> fetchedFeeds = feed.getEntries();
+            rssFeedBuffer.addAll(fetchedFeeds);
+        }
+    }
+}
+
+class FetcherEventListenerImpl implements FetcherListener {
+
+    private RSSRecordReader reader;
+    private Logger LOGGER;
+
+    public FetcherEventListenerImpl(RSSRecordReader reader, Logger LOGGER) {
+        this.reader = reader;
+        this.LOGGER = LOGGER;
+    }
+
+    /**
+     * @see com.sun.syndication.fetcher.FetcherListener#fetcherEvent(com.sun.syndication.fetcher.FetcherEvent)
+     */
+    @Override
+    public void fetcherEvent(FetcherEvent event) {
+        String eventType = event.getEventType();
+        if (FetcherEvent.EVENT_TYPE_FEED_POLLED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Polled. URL = " + event.getUrlString());
+            }
+        } else if (FetcherEvent.EVENT_TYPE_FEED_RETRIEVED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Retrieved. URL = " + event.getUrlString());
+            }
+            (reader).setModified(true);
+        } else if (FetcherEvent.EVENT_TYPE_FEED_UNCHANGED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Unchanged. URL = " + event.getUrlString());
+            }
+            (reader).setModified(true);
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java
new file mode 100644
index 0000000..9864805
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataExceptionUtils;
+
+public class SemiStructuredRecordReader extends AbstractStreamRecordReader {
+
+    private int depth;
+    private boolean prevCharEscape;
+    private boolean inString;
+    private char recordStart;
+    private char recordEnd;
+    private int recordNumber = 0;
+
+    public int getRecordNumber() {
+        return recordNumber;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        String recStartString = configuration.get(ExternalDataConstants.KEY_RECORD_START);
+        String recEndString = configuration.get(ExternalDataConstants.KEY_RECORD_END);
+        if (recStartString != null) {
+            if (recStartString.length() != 1) {
+                throw new AsterixException(
+                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_START,
+                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recStartString));
+            }
+            recordStart = recStartString.charAt(0);
+        } else {
+            recordStart = ExternalDataConstants.DEFAULT_RECORD_START;
+        }
+        if (recEndString != null) {
+            if (recEndString.length() != 1) {
+                throw new AsterixException(
+                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_END,
+                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recEndString));
+            }
+            recordEnd = recEndString.charAt(0);
+        } else {
+            recordEnd = ExternalDataConstants.DEFAULT_RECORD_END;
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        record.reset();
+        boolean hasStarted = false;
+        boolean hasFinished = false;
+        prevCharEscape = false;
+        inString = false;
+        depth = 0;
+        do {
+            int startPosn = bufferPosn; //starting from where we left off the last time
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    return false; // EOF
+                }
+            }
+            if (!hasStarted) {
+                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
+                    if (inputBuffer[bufferPosn] == recordStart) {
+                        startPosn = bufferPosn;
+                        hasStarted = true;
+                        depth = 1;
+                        ++bufferPosn; // at next invocation proceed from following byte
+                        break;
+                    } else if (inputBuffer[bufferPosn] != ExternalDataConstants.SPACE
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.TAB
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.LF
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.CR) {
+                        // corrupted file. clear the buffer and stop reading
+                        reader.skipError();
+                        bufferPosn = bufferLength = 0;
+                        throw new IOException("Malformed input stream");
+                    }
+                }
+            }
+            if (hasStarted) {
+                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
+                    if (inString) {
+                        // we are in a string, we only care about the string end
+                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE && !prevCharEscape) {
+                            inString = false;
+                        }
+                        if (prevCharEscape) {
+                            prevCharEscape = false;
+                        } else {
+                            prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                        }
+                    } else {
+                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE) {
+                            inString = true;
+                        } else if (inputBuffer[bufferPosn] == recordStart) {
+                            depth += 1;
+                        } else if (inputBuffer[bufferPosn] == recordEnd) {
+                            depth -= 1;
+                            if (depth == 0) {
+                                hasFinished = true;
+                                ++bufferPosn; // at next invocation proceed from following byte
+                                break;
+                            }
+                        }
+                    }
+                }
+            }
+
+            int appendLength = bufferPosn - startPosn;
+            if (appendLength > 0) {
+                record.append(inputBuffer, startPosn, appendLength);
+            }
+        } while (!hasFinished);
+        record.endRecord();
+        recordNumber++;
+        return true;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java
new file mode 100644
index 0000000..c294ccb
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+
+public class SequenceLookupReader extends AbstractCharRecordLookupReader {
+
+    public SequenceLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(SequenceLookupReader.class.getName());
+    private Reader reader;
+    private Writable key;
+
+    @Override
+    protected void readRecord(RecordId rid) throws IOException {
+        reader.seek(rid.getOffset());
+        reader.next(key, value);
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file ", e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        reader = new SequenceFile.Reader(fs, new Path(file.getFileName()), conf);
+        key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+        value = (Text) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java
new file mode 100644
index 0000000..b276bfa
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+public class TextLookupReader extends AbstractCharRecordLookupReader {
+
+    public TextLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(TextLookupReader.class.getName());
+    private HDFSTextLineReader reader;
+
+    @Override
+    protected void readRecord(RecordId rid) throws IOException {
+        reader.seek(rid.getOffset());
+        reader.readLine(value);
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file ", e);
+        }
+    }
+
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        if (reader == null) {
+            reader = new HDFSTextLineReader();
+        }
+        reader.resetReader(fs.open(new Path(file.getFileName())));;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java
new file mode 100644
index 0000000..34d8122
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.util.TwitterUtil;
+import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import twitter4j.Query;
+import twitter4j.QueryResult;
+import twitter4j.Status;
+import twitter4j.Twitter;
+import twitter4j.TwitterException;
+
+public class TwitterPullRecordReader implements IRecordReader<Status> {
+
+    private String keywords;
+    private Query query;
+    private Twitter twitter;
+    private int requestInterval = 5; // seconds
+    private QueryResult result;
+    private int nextTweetIndex = 0;
+    private long lastTweetIdReceived = 0;
+    private GenericRecord<Status> record;
+
+    public TwitterPullRecordReader() {
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        twitter = TwitterUtil.getTwitterService(configuration);
+        keywords = configuration.get(SearchAPIConstants.QUERY);
+        requestInterval = Integer.parseInt(configuration.get(SearchAPIConstants.INTERVAL));
+        query = new Query(keywords);
+        query.setCount(100);
+        record = new GenericRecord<Status>();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return true;
+    }
+
+    @Override
+    public IRawRecord<Status> next() throws IOException, InterruptedException {
+        if (result == null || nextTweetIndex >= result.getTweets().size()) {
+            Thread.sleep(1000 * requestInterval);
+            query.setSinceId(lastTweetIdReceived);
+            try {
+                result = twitter.search(query);
+            } catch (TwitterException e) {
+                throw new HyracksDataException(e);
+            }
+            nextTweetIndex = 0;
+        }
+        if (result != null && !result.getTweets().isEmpty()) {
+            List<Status> tw = result.getTweets();
+            Status tweet = tw.get(nextTweetIndex++);
+            if (lastTweetIdReceived < tweet.getId()) {
+                lastTweetIdReceived = tweet.getId();
+            }
+            record.set(tweet);
+            return record;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public Class<Status> getRecordClass() throws IOException {
+        return Status.class;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java
new file mode 100644
index 0000000..e7c141d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.util.TwitterUtil;
+
+import twitter4j.FilterQuery;
+import twitter4j.StallWarning;
+import twitter4j.Status;
+import twitter4j.StatusDeletionNotice;
+import twitter4j.StatusListener;
+import twitter4j.TwitterStream;
+
+public class TwitterPushRecordReader implements IRecordReader<Status> {
+    private LinkedBlockingQueue<Status> inputQ;
+    private TwitterStream twitterStream;
+    private GenericRecord<Status> record;
+
+    @Override
+    public void close() throws IOException {
+        twitterStream.clearListeners();
+        twitterStream.cleanUp();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new GenericRecord<Status>();
+        inputQ = new LinkedBlockingQueue<Status>();
+        twitterStream = TwitterUtil.getTwitterStream(configuration);
+        twitterStream.addListener(new TweetListener(inputQ));
+        FilterQuery query = TwitterUtil.getFilterQuery(configuration);
+        if (query != null) {
+            twitterStream.filter(query);
+        } else {
+            twitterStream.sample();
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return true;
+    }
+
+    @Override
+    public IRawRecord<Status> next() throws IOException, InterruptedException {
+        Status tweet = inputQ.poll();
+        if (tweet == null) {
+            return null;
+        }
+        record.set(tweet);
+        return record;
+    }
+
+    @Override
+    public Class<? extends Status> getRecordClass() throws IOException {
+        return Status.class;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    private class TweetListener implements StatusListener {
+
+        private LinkedBlockingQueue<Status> inputQ;
+
+        public TweetListener(LinkedBlockingQueue<Status> inputQ) {
+            this.inputQ = inputQ;
+        }
+
+        @Override
+        public void onStatus(Status tweet) {
+            inputQ.add(tweet);
+        }
+
+        @Override
+        public void onException(Exception arg0) {
+
+        }
+
+        @Override
+        public void onDeletionNotice(StatusDeletionNotice arg0) {
+        }
+
+        @Override
+        public void onScrubGeo(long arg0, long arg1) {
+        }
+
+        @Override
+        public void onStallWarning(StallWarning arg0) {
+        }
+
+        @Override
+        public void onTrackLimitationNotice(int arg0) {
+        }
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java
new file mode 100644
index 0000000..e9fad25
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.ILookupReaderFactory;
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.input.record.reader.RCLookupReader;
+import org.apache.asterix.external.input.record.reader.SequenceLookupReader;
+import org.apache.asterix.external.input.record.reader.TextLookupReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.hdfs.dataflow.ConfFactory;
+
+public class HDFSLookupReaderFactory<T> implements ILookupReaderFactory<T> {
+
+    protected static final long serialVersionUID = 1L;
+    protected transient AlgebricksPartitionConstraint clusterLocations;
+    protected ConfFactory confFactory;
+    protected Map<String, String> configuration;
+
+    public HDFSLookupReaderFactory() {
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        clusterLocations = HDFSUtils.getPartitionConstraints(clusterLocations);
+        return clusterLocations;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
+        confFactory = new ConfFactory(conf);
+
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
+            ExternalFileIndexAccessor snapshotAccessor) throws Exception {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        JobConf conf = confFactory.getConf();
+        FileSystem fs = FileSystem.get(conf);
+        switch (inputFormatParameter) {
+            case ExternalDataConstants.INPUT_FORMAT_TEXT:
+                return (ILookupRecordReader<? extends T>) new TextLookupReader(snapshotAccessor, fs, conf);
+            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
+                return (ILookupRecordReader<? extends T>) new SequenceLookupReader(snapshotAccessor, fs, conf);
+            case ExternalDataConstants.INPUT_FORMAT_RC:
+                return (ILookupRecordReader<? extends T>) new RCLookupReader(snapshotAccessor, fs, conf);
+            default:
+                throw new AsterixException("Unrecognised input format: " + inputFormatParameter);
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java
new file mode 100644
index 0000000..05d419d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.reader.AbstractStreamRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.LineRecordReader;
+import org.apache.asterix.external.input.record.reader.QuotedLineRecordReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class LineRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        LineRecordReader recordReader;
+        if (quoteString != null) {
+            recordReader = new QuotedLineRecordReader();
+        } else {
+            recordReader = new LineRecordReader();
+        }
+        return configureReader(recordReader, ctx, partition);
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java
new file mode 100644
index 0000000..a672f2f
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.factory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.RSSRecordReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+
+public class RSSRecordReaderFactory implements IRecordReaderFactory<SyndEntryImpl> {
+
+    private static final long serialVersionUID = 1L;
+    private Map<String, String> configuration;
+    private List<String> urls = new ArrayList<String>();
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(urls.size());
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        String url = configuration.get(ExternalDataConstants.KEY_RSS_URL);
+        if (url == null) {
+            throw new IllegalArgumentException("no RSS URL provided");
+        }
+        initializeURLs(url);
+    }
+
+    private void initializeURLs(String url) {
+        urls.clear();
+        String[] rssURLs = url.split(",");
+        for (String rssURL : rssURLs) {
+            urls.add(rssURL);
+        }
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @Override
+    public IRecordReader<? extends SyndEntryImpl> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws Exception {
+        RSSRecordReader reader = new RSSRecordReader(urls.get(partition));
+        reader.configure(configuration);
+        return reader;
+    }
+
+    @Override
+    public Class<? extends SyndEntryImpl> getRecordClass() {
+        return SyndEntryImpl.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java
new file mode 100644
index 0000000..91b439c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.reader.AbstractStreamRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.SemiStructuredRecordReader;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class SemiStructuredRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
+        return configureReader(recordReader, ctx, partition);
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java
new file mode 100644
index 0000000..72aaa37
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.factory;
+
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.TwitterPullRecordReader;
+import org.apache.asterix.external.input.record.reader.TwitterPushRecordReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.TwitterUtil;
+import org.apache.asterix.external.util.TwitterUtil.AuthenticationConstants;
+import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import twitter4j.Status;
+
+public class TwitterRecordReaderFactory implements IRecordReaderFactory<Status> {
+
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(TwitterRecordReaderFactory.class.getName());
+
+    private static final String DEFAULT_INTERVAL = "10"; // 10 seconds
+    private static final int INTAKE_CARDINALITY = 1; // degree of parallelism at intake stage
+
+    private Map<String, String> configuration;
+    private boolean pull;
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(INTAKE_CARDINALITY);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
+        if (!validateConfiguration(configuration)) {
+            StringBuilder builder = new StringBuilder();
+            builder.append("One or more parameters are missing from adapter configuration\n");
+            builder.append(AuthenticationConstants.OAUTH_CONSUMER_KEY + "\n");
+            builder.append(AuthenticationConstants.OAUTH_CONSUMER_SECRET + "\n");
+            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN + "\n");
+            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET + "\n");
+            throw new Exception(builder.toString());
+        }
+        if (ExternalDataUtils.isPull(configuration)) {
+            pull = true;
+            if (configuration.get(SearchAPIConstants.QUERY) == null) {
+                throw new AsterixException(
+                        "parameter " + SearchAPIConstants.QUERY + " not specified as part of adaptor configuration");
+            }
+            String interval = configuration.get(SearchAPIConstants.INTERVAL);
+            if (interval != null) {
+                try {
+                    Integer.parseInt(interval);
+                } catch (NumberFormatException nfe) {
+                    throw new IllegalArgumentException(
+                            "parameter " + SearchAPIConstants.INTERVAL + " is defined incorrectly, expecting a number");
+                }
+            } else {
+                configuration.put(SearchAPIConstants.INTERVAL, DEFAULT_INTERVAL);
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning(" Parameter " + SearchAPIConstants.INTERVAL + " not defined, using default ("
+                            + DEFAULT_INTERVAL + ")");
+                }
+            }
+        } else if (ExternalDataUtils.isPush(configuration)) {
+            pull = false;
+        } else {
+            throw new AsterixException("One of boolean parameters " + ExternalDataConstants.KEY_PULL + " and "
+                    + ExternalDataConstants.KEY_PUSH + "must be specified as part of adaptor configuration");
+        }
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @Override
+    public IRecordReader<? extends Status> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        IRecordReader<Status> reader;
+        if (pull) {
+            reader = new TwitterPullRecordReader();
+        } else {
+            reader = new TwitterPushRecordReader();
+        }
+        reader.configure(configuration);
+        return reader;
+    }
+
+    @Override
+    public Class<? extends Status> getRecordClass() {
+        return Status.class;
+    }
+
+    private boolean validateConfiguration(Map<String, String> configuration) {
+        String consumerKey = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_KEY);
+        String consumerSecret = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_SECRET);
+        String accessToken = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN);
+        String tokenSecret = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET);
+        if (consumerKey == null || consumerSecret == null || accessToken == null || tokenSecret == null) {
+            return false;
+        }
+        return true;
+    }
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
similarity index 76%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
index f602656..73f6195 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePreprocessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
@@ -16,11 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.input.stream;
 
-import java.nio.ByteBuffer;
+import java.io.InputStream;
 
-public interface IFramePreprocessor {
+public abstract class AInputStream extends InputStream {
+    public abstract boolean skipError() throws Exception;
 
-    public void preProcess(ByteBuffer frame) throws Exception;
+    public abstract boolean stop() throws Exception;
+
 }
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
similarity index 69%
copy from asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
index 1dfbee9..e573f74 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/feeds/IFramePostProcessor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
@@ -16,13 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.feeds;
+package org.apache.asterix.external.input.stream;
 
-import java.nio.ByteBuffer;
+import java.io.InputStreamReader;
 
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+public class AInputStreamReader extends InputStreamReader {
+    private AInputStream in;
 
-public interface IFramePostProcessor {
+    public AInputStreamReader(AInputStream in) {
+        super(in);
+        this.in = in;
+    }
 
-    public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor);
+    public boolean skipError() throws Exception {
+        return in.skipError();
+    }
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/BasicInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/BasicInputStream.java
new file mode 100644
index 0000000..aa7a3d8
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/BasicInputStream.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class BasicInputStream extends AInputStream {
+    private final InputStream in;
+
+    public BasicInputStream(InputStream in) {
+        this.in = in;
+    }
+
+    @Override
+    public int read() throws IOException {
+        return in.read();
+    }
+
+    @Override
+    public int read(byte b[]) throws IOException {
+        return in.read(b);
+    }
+
+    @Override
+    public int read(byte b[], int off, int len) throws IOException {
+        return in.read(b, off, len);
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        return in.skip(n);
+
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public synchronized void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public synchronized void reset() throws IOException {
+        in.reset();
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public boolean skipError() {
+        return false;
+    }
+
+    @Override
+    public boolean stop() throws Exception {
+        return false;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java
new file mode 100644
index 0000000..b3ad1c3
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.external.provider.ExternalIndexerProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+public class HDFSInputStreamProvider<K> extends HDFSRecordReader<K, Text> implements IInputStreamProvider {
+
+    public HDFSInputStreamProvider(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf, Map<String, String> configuration, List<ExternalFile> snapshot) throws Exception {
+        super(read, inputSplits, readSchedule, nodeName, conf);
+        value = new Text();
+        configure(configuration);
+        if (snapshot != null) {
+            setSnapshot(snapshot);
+            setIndexer(ExternalIndexerProvider.getIndexer(configuration));
+            if (currentSplitIndex < snapshot.size()) {
+                indexer.reset(this);
+            }
+        }
+    }
+
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        return new HDFSInputStream();
+    }
+
+    private class HDFSInputStream extends AInputStream {
+        int pos = 0;
+
+        @Override
+        public int read() throws IOException {
+            if (value.getLength() < pos) {
+                if (!readMore()) {
+                    return -1;
+                }
+            } else if (value.getLength() == pos) {
+                pos++;
+                return ExternalDataConstants.EOL;
+            }
+            return value.getBytes()[pos++];
+        }
+
+        private int readRecord(byte[] buffer, int offset, int len) {
+            int actualLength = value.getLength() + 1;
+            if ((actualLength - pos) > len) {
+                //copy partial record
+                System.arraycopy(value.getBytes(), pos, buffer, offset, len);
+                pos += len;
+                return len;
+            } else {
+                int numBytes = value.getLength() - pos;
+                System.arraycopy(value.getBytes(), pos, buffer, offset, numBytes);
+                buffer[offset + numBytes] = ExternalDataConstants.LF;
+                pos += numBytes;
+                numBytes++;
+                return numBytes;
+            }
+        }
+
+        @Override
+        public int read(byte[] buffer, int offset, int len) throws IOException {
+            if (value.getLength() > pos) {
+                return readRecord(buffer, offset, len);
+            }
+            if (!readMore()) {
+                return -1;
+            }
+            return readRecord(buffer, offset, len);
+        }
+
+        private boolean readMore() throws IOException {
+            try {
+                pos = 0;
+                return HDFSInputStreamProvider.this.hasNext();
+            } catch (Exception e) {
+                throw new IOException(e);
+            }
+        }
+
+        @Override
+        public boolean skipError() throws Exception {
+            return true;
+        }
+
+        @Override
+        public boolean stop() throws Exception {
+            return false;
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java
new file mode 100644
index 0000000..b511617
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.dataflow.std.file.FileSplit;
+
+public class LocalFSInputStreamProvider implements IInputStreamProvider {
+
+    private FileSplit[] fileSplits;
+    private int partition;
+
+    public LocalFSInputStreamProvider(FileSplit[] fileSplits, IHyracksTaskContext ctx,
+            Map<String, String> configuration, int partition) {
+        this.partition = partition;
+        this.fileSplits = fileSplits;
+    }
+
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        FileSplit split = fileSplits[partition];
+        File inputFile = split.getLocalFile().getFile();
+        InputStream in;
+        try {
+            in = new FileInputStream(inputFile);
+            return new BasicInputStream(in);
+        } catch (FileNotFoundException e) {
+            throw new IOException(e);
+        }
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStream.java
new file mode 100644
index 0000000..2253a73
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStream.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+
+public class SocketInputStream extends AInputStream {
+    private ServerSocket server;
+    private Socket socket;
+    private InputStream connectionStream;
+
+    public SocketInputStream(ServerSocket server) throws IOException {
+        this.server = server;
+        socket = server.accept();
+        connectionStream = socket.getInputStream();
+    }
+
+    @Override
+    public int read() throws IOException {
+        int read = connectionStream.read();
+        while (read < 0) {
+            accept();
+            read = connectionStream.read();
+        }
+        return read;
+    }
+
+    @Override
+    public boolean skipError() throws Exception {
+        accept();
+        return true;
+    }
+
+    @Override
+    public int read(byte b[]) throws IOException {
+        int read = connectionStream.read(b, 0, b.length);
+        while (read < 0) {
+            accept();
+            read = connectionStream.read(b, 0, b.length);
+        }
+        return read;
+    }
+
+    @Override
+    public int read(byte b[], int off, int len) throws IOException {
+        int read = connectionStream.read(b, off, len);
+        while (read < 0) {
+            accept();
+            read = connectionStream.read(b, off, len);
+        }
+        return read;
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        return 0;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return 1;
+    }
+
+    @Override
+    public void close() throws IOException {
+        connectionStream.close();
+        socket.close();
+        server.close();
+    }
+
+    private void accept() throws IOException {
+        connectionStream.close();
+        socket.close();
+        socket = server.accept();
+        connectionStream = socket.getInputStream();
+    }
+
+    @Override
+    public boolean stop() throws Exception {
+        return false;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
similarity index 63%
copy from asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java
copy to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
index 9358a52..1f920e9 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/IFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
@@ -16,14 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.adapter.factory;
+package org.apache.asterix.external.input.stream;
 
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
+import java.net.ServerSocket;
 
-public interface IFeedAdapterFactory extends IAdapterFactory {
+import org.apache.asterix.external.api.IInputStreamProvider;
 
-    public boolean isRecordTrackingEnabled();
+public class SocketInputStreamProvider implements IInputStreamProvider {
+    private ServerSocket server;
 
-    public IIntakeProgressTracker createIntakeProgressTracker();
+    public SocketInputStreamProvider(ServerSocket server) {
+        this.server = server;
+    }
 
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        return new SocketInputStream(server);
+    }
 }
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
similarity index 63%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
index fffbc17..d32a94f 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.input.stream;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -29,66 +29,82 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.external.dataset.adapter.StreamBasedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.runtime.TweetGenerator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
-/**
- * A simulator of the Twitter Firehose. Generates meaningful tweets
- * at a configurable rate
- */
-public class TwitterFirehoseFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+public class TwitterFirehoseInputStreamProvider implements IInputStreamProvider {
 
-    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseInputStreamProvider.class.getName());
 
-    private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseFeedAdapter.class.getName());
+    private ExecutorService executorService;
 
-    private ExecutorService executorService = Executors.newCachedThreadPool();
+    private PipedOutputStream outputStream;
 
-    private PipedOutputStream outputStream = new PipedOutputStream();
+    private PipedInputStream inputStream;
 
-    private PipedInputStream inputStream = new PipedInputStream(outputStream);
+    private TwitterServer twitterServer;
 
-    private final TwitterServer twitterServer;
-
-    public TwitterFirehoseFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
-            ARecordType outputtype, IHyracksTaskContext ctx, int partition) throws Exception {
-        super(parserFactory, outputtype, ctx, partition);
-        this.twitterServer = new TwitterServer(configuration, partition, outputtype, outputStream, executorService);
+    public TwitterFirehoseInputStreamProvider(Map<String, String> configuration, IHyracksTaskContext ctx, int partition)
+            throws Exception {
+        executorService = Executors.newCachedThreadPool();
+        outputStream = new PipedOutputStream();
+        inputStream = new PipedInputStream(outputStream);
+        twitterServer = new TwitterServer(configuration, partition, outputStream, executorService, inputStream);
     }
 
     @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
+    public AInputStream getInputStream() throws Exception {
         twitterServer.start();
-        super.start(partition, writer);
+        return twitterServer;
     }
 
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        return inputStream;
-    }
-
-    private static class TwitterServer {
+    private static class TwitterServer extends AInputStream {
         private final DataProvider dataProvider;
         private final ExecutorService executorService;
+        private InputStream in;
+        private boolean started;
 
-        public TwitterServer(Map<String, String> configuration, int partition, ARecordType outputtype, OutputStream os,
-                ExecutorService executorService) throws Exception {
-            dataProvider = new DataProvider(configuration, outputtype, partition, os);
+        public TwitterServer(Map<String, String> configuration, int partition, OutputStream os,
+                ExecutorService executorService, InputStream in) throws Exception {
+            dataProvider = new DataProvider(configuration, partition, os);
             this.executorService = executorService;
+            this.in = in;
+            this.started = false;
         }
 
-        public void stop() throws IOException {
+        @Override
+        public boolean stop() throws IOException {
             dataProvider.stop();
+            return true;
         }
 
         public void start() {
             executorService.execute(dataProvider);
         }
 
+        @Override
+        public boolean skipError() throws Exception {
+            return false;
+        }
+
+        @Override
+        public int read() throws IOException {
+            if (!started) {
+                start();
+                started = true;
+            }
+            return in.read();
+        }
+
+        @Override
+        public int read(byte b[], int off, int len) throws IOException {
+            if (!started) {
+                start();
+                started = true;
+            }
+            return in.read(b, off, len);
+        }
     }
 
     private static class DataProvider implements Runnable {
@@ -106,8 +122,7 @@
             CONTROLLED
         }
 
-        public DataProvider(Map<String, String> configuration, ARecordType outputtype, int partition, OutputStream os)
-                throws Exception {
+        public DataProvider(Map<String, String> configuration, int partition, OutputStream os) throws Exception {
             this.tweetGenerator = new TweetGenerator(configuration, partition);
             this.tweetGenerator.registerSubscriber(os);
             this.os = os;
@@ -165,27 +180,4 @@
         }
 
     }
-
-    @Override
-    public void stop() throws Exception {
-        twitterServer.stop();
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PUSH;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        try {
-            twitterServer.stop();
-        } catch (Exception re) {
-            re.printStackTrace();
-            return false;
-        }
-        twitterServer.start();
-        return true;
-    }
-
-}
\ No newline at end of file
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
similarity index 60%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
index 251d69a..14c712a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
@@ -16,72 +16,57 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.adapter.factory;
+package org.apache.asterix.external.input.stream.factory;
 
 import java.io.File;
-import java.util.List;
 import java.util.Map;
 import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter;
-import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.INodeResolver;
+import org.apache.asterix.external.api.INodeResolverFactory;
+import org.apache.asterix.external.input.stream.LocalFSInputStreamProvider;
 import org.apache.asterix.external.util.DNSResolverFactory;
-import org.apache.asterix.external.util.INodeResolver;
-import org.apache.asterix.external.util.INodeResolverFactory;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 
-/**
- * Factory class for creating an instance of NCFileSystemAdapter. An
- * NCFileSystemAdapter reads external data residing on the local file system of
- * an NC.
- */
-public class NCFileSystemAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
+public class LocalFSInputStreamProviderFactory implements IInputStreamProviderFactory {
+
     private static final long serialVersionUID = 1L;
 
-    public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
-
-    private static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
-
-    private IAType sourceDatatype;
-    private FileSplit[] fileSplits;
-    private ARecordType outputType;
+    protected static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
+    protected static final Logger LOGGER = Logger.getLogger(LocalFSInputStreamProviderFactory.class.getName());
+    protected static INodeResolver nodeResolver;
+    protected Map<String, String> configuration;
+    protected FileSplit[] fileSplits;
 
     @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(fileSplits, parserFactory, sourceDatatype, ctx);
-        return fsAdapter;
+    public IInputStreamProvider createInputStreamProvider(IHyracksTaskContext ctx, int partition) throws Exception {
+        return new LocalFSInputStreamProvider(fileSplits, ctx, configuration, partition);
     }
 
     @Override
-    public String getName() {
-        return NC_FILE_SYSTEM_ADAPTER_NAME;
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.STREAM;
     }
 
     @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
+    public boolean isIndexible() {
+        return false;
     }
 
     @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+    public void configure(Map<String, String> configuration) throws Exception {
         this.configuration = configuration;
-        this.outputType = outputType;
-        String[] splits = configuration.get(AsterixTupleParserFactory.KEY_PATH).split(",");
-        IAType sourceDatatype = outputType;
+        String[] splits = configuration.get(ExternalDataConstants.KEY_PATH).split(",");
         configureFileSplits(splits);
-        configureFormat(sourceDatatype);
-
     }
 
     @Override
@@ -122,15 +107,18 @@
 
     protected INodeResolver getNodeResolver() {
         if (nodeResolver == null) {
-            nodeResolver = initializeNodeResolver();
+            synchronized (DEFAULT_NODE_RESOLVER) {
+                if (nodeResolver == null) {
+                    nodeResolver = initializeNodeResolver();
+                }
+            }
         }
         return nodeResolver;
     }
 
     private static INodeResolver initializeNodeResolver() {
         INodeResolver nodeResolver = null;
-        String configuredNodeResolverFactory = System
-                .getProperty(AsterixTupleParserFactory.NODE_RESOLVER_FACTORY_PROPERTY);
+        String configuredNodeResolverFactory = System.getProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY);
         if (configuredNodeResolverFactory != null) {
             try {
                 nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
@@ -148,19 +136,4 @@
         }
         return nodeResolver;
     }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
-    }
-
-    @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.UNKNOWN;
-    }
-
-    public void setFiles(List<ExternalFile> files) throws AlgebricksException {
-        throw new AlgebricksException("can't set files for this Adapter");
-    }
-
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
new file mode 100644
index 0000000..37afa53
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream.factory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.input.stream.SocketInputStreamProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.util.AsterixRuntimeUtil;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class SocketInputStreamProviderFactory implements IInputStreamProviderFactory {
+
+    private static final long serialVersionUID = 1L;
+    private List<Pair<String, Integer>> sockets;
+    private Mode mode = Mode.IP;
+
+    public static enum Mode {
+        NC,
+        IP
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        sockets = new ArrayList<Pair<String, Integer>>();
+        String modeValue = configuration.get(ExternalDataConstants.KEY_MODE);
+        if (modeValue != null) {
+            mode = Mode.valueOf(modeValue.trim().toUpperCase());
+        }
+        String socketsValue = configuration.get(ExternalDataConstants.KEY_SOCKETS);
+        if (socketsValue == null) {
+            throw new IllegalArgumentException("\'sockets\' parameter not specified as part of adapter configuration");
+        }
+        Map<InetAddress, Set<String>> ncMap = AsterixRuntimeUtil.getNodeControllerMap();
+        List<String> ncs = AsterixRuntimeUtil.getAllNodeControllers();
+        String[] socketsArray = socketsValue.split(",");
+        Random random = new Random();
+        for (String socket : socketsArray) {
+            String[] socketTokens = socket.split(":");
+            String host = socketTokens[0].trim();
+            int port = Integer.parseInt(socketTokens[1].trim());
+            Pair<String, Integer> p = null;
+            switch (mode) {
+                case IP:
+                    Set<String> ncsOnIp = ncMap.get(InetAddress.getByName(host));
+                    if (ncsOnIp == null || ncsOnIp.isEmpty()) {
+                        throw new IllegalArgumentException("Invalid host " + host
+                                + " as it is not part of the AsterixDB cluster. Valid choices are "
+                                + StringUtils.join(ncMap.keySet(), ", "));
+                    }
+                    String[] ncArray = ncsOnIp.toArray(new String[] {});
+                    String nc = ncArray[random.nextInt(ncArray.length)];
+                    p = new Pair<String, Integer>(nc, port);
+                    break;
+
+                case NC:
+                    p = new Pair<String, Integer>(host, port);
+                    if (!ncs.contains(host)) {
+                        throw new IllegalArgumentException(
+                                "Invalid NC " + host + " as it is not part of the AsterixDB cluster. Valid choices are "
+                                        + StringUtils.join(ncs, ", "));
+
+                    }
+                    break;
+            }
+            sockets.add(p);
+        }
+    }
+
+    @Override
+    public synchronized IInputStreamProvider createInputStreamProvider(IHyracksTaskContext ctx, int partition)
+            throws IOException, AsterixException {
+        Pair<String, Integer> socket = sockets.get(partition);
+        ServerSocket server = new ServerSocket(socket.second);
+        return new SocketInputStreamProvider(server);
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() {
+        List<String> locations = new ArrayList<String>();
+        for (Pair<String, Integer> socket : sockets) {
+            locations.add(socket.first);
+        }
+        return new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[] {}));
+    }
+
+    public List<Pair<String, Integer>> getSockets() {
+        return sockets;
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.STREAM;
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+}
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamProviderFactory.java
similarity index 63%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamProviderFactory.java
index f7e79f7..b86c294 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamProviderFactory.java
@@ -16,20 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.input.stream.factory;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
-import org.apache.asterix.external.adapter.factory.StreamBasedAdapterFactory;
-import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
 import org.apache.asterix.om.util.AsterixClusterProperties;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -40,7 +35,7 @@
  * configurable rate measured in terms of TPS (tweets/second). The stream of
  * tweets lasts for a configurable duration (measured in seconds).
  */
-public class TwitterFirehoseFeedAdapterFactory extends StreamBasedAdapterFactory implements IFeedAdapterFactory {
+public class TwitterFirehoseStreamProviderFactory implements IInputStreamProviderFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -55,25 +50,7 @@
      **/
     private static final String KEY_INGESTION_LOCATIONS = "ingestion-location";
 
-    private ARecordType outputType;
-
-    @Override
-    public String getName() {
-        return "twitter_firehose";
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        configuration.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_ADM);
-        this.configuration = configuration;
-        this.outputType = outputType;
-        this.configureFormat(outputType);
-    }
+    private Map<String, String> configuration;
 
     @Override
     public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
@@ -98,28 +75,22 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        return new TwitterFirehoseFeedAdapter(configuration, parserFactory, outputType, ctx, partition);
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.STREAM;
     }
 
     @Override
-    public ARecordType getAdapterOutputType() {
-        return outputType;
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
     }
 
     @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.ADM;
-    }
-
-    @Override
-    public boolean isRecordTrackingEnabled() {
+    public boolean isIndexible() {
         return false;
     }
 
     @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        throw new UnsupportedOperationException("Tracking of ingested records not enabled");
+    public IInputStreamProvider createInputStreamProvider(IHyracksTaskContext ctx, int partition) throws Exception {
+        return null;
     }
-
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
index 14f831b..e9c15cb 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
@@ -21,6 +21,9 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
index d0d44e3..d10ff6d 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.library;
 
+import org.apache.asterix.external.api.IExternalFunction;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/JTypeObjectFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/JTypeObjectFactory.java
index 677ed76..cdd29ff 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/JTypeObjectFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/JTypeObjectFactory.java
@@ -20,7 +20,7 @@
 
 import java.util.List;
 
-import org.apache.asterix.external.library.java.IJObject;
+import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.external.library.java.JObjects.JBoolean;
 import org.apache.asterix.external.library.java.JObjects.JCircle;
 import org.apache.asterix.external.library.java.JObjects.JDate;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
index dc80fda..13ea589 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
@@ -19,7 +19,8 @@
 package org.apache.asterix.external.library;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.library.java.IJObject;
+import org.apache.asterix.external.api.IFunctionHelper;
+import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.external.library.java.JObjectPointableVisitor;
 import org.apache.asterix.external.library.java.JObjects.JNull;
 import org.apache.asterix.external.library.java.JTypeTag;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ResultCollector.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ResultCollector.java
index 2671f13..192bd4e 100755
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/ResultCollector.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/ResultCollector.java
@@ -21,6 +21,7 @@
 import java.io.DataOutput;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IResultCollector;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableDouble;
 import org.apache.asterix.om.base.AMutableFloat;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java
index 062bfd7..453cf39 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.java.IJObject;
+import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.container.IObjectPool;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
index 29ce728..1835739 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
@@ -43,6 +43,10 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.external.api.IJListAccessor;
+import org.apache.asterix.external.api.IJObject;
+import org.apache.asterix.external.api.IJObjectAccessor;
+import org.apache.asterix.external.api.IJRecordAccessor;
 import org.apache.asterix.external.library.TypeInfo;
 import org.apache.asterix.external.library.java.JObjects.JBoolean;
 import org.apache.asterix.external.library.java.JObjects.JByte;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
index e8bef5a..1a75115 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
@@ -22,6 +22,10 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IJListAccessor;
+import org.apache.asterix.external.api.IJObject;
+import org.apache.asterix.external.api.IJObjectAccessor;
+import org.apache.asterix.external.api.IJRecordAccessor;
 import org.apache.asterix.external.library.TypeInfo;
 import org.apache.asterix.external.library.java.JObjectAccessors.JListAccessor;
 import org.apache.asterix.external.library.java.JObjectAccessors.JRecordAccessor;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
index 146a247..93b4bf1 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
+import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.external.library.java.JObjects.ByteArrayAccessibleDataInputStream;
 import org.apache.asterix.external.library.java.JObjects.JBoolean;
 import org.apache.asterix.external.library.java.JObjects.JCircle;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
index e9a63b2..42b0742 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
@@ -53,6 +53,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AFloat;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
similarity index 87%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
index 89abf0f..371c603 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.util.List;
 
@@ -39,7 +39,8 @@
 // 1. commit transaction operation
 // 2. abort transaction operation
 // 3. recover transaction operation
-public abstract class AbstractExternalDatasetIndexesOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+public abstract class AbstractExternalDatasetIndexesOperatorDescriptor
+        extends AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
     private ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory;
@@ -66,9 +67,8 @@
     }
 
     // opening and closing the index is done inside these methods since we don't always need open indexes
-    protected abstract void performOpOnIndex(
-            IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
-            IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception;
+    protected abstract void performOpOnIndex(IIndexDataflowHelperFactory indexDataflowHelperFactory,
+            IHyracksTaskContext ctx, IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception;
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
@@ -79,21 +79,20 @@
             public void initialize() throws HyracksDataException {
                 try {
                     // only in partition of device id = 0, we perform the operation on the files index
-                    if(fileIndexInfo.getFileSplitProvider().getFileSplits()[partition].getIODeviceId() == 0){
+                    if (fileIndexInfo.getFileSplitProvider().getFileSplits()[partition].getIODeviceId() == 0) {
                         performOpOnIndex(filesIndexDataflowHelperFactory, ctx, fileIndexInfo, partition);
                     }
                     // perform operation on btrees
                     for (int i = 0; i < bTreeIndexesDataflowHelperFactories.size(); i++) {
-                        performOpOnIndex(bTreeIndexesDataflowHelperFactories.get(i), ctx,
-                                bTreeIndexesInfos.get(i), partition);
+                        performOpOnIndex(bTreeIndexesDataflowHelperFactories.get(i), ctx, bTreeIndexesInfos.get(i),
+                                partition);
                     }
                     // perform operation on rtrees
                     for (int i = 0; i < rTreeIndexesDataflowHelperFactories.size(); i++) {
-                        performOpOnIndex(rTreeIndexesDataflowHelperFactories.get(i), ctx,
-                                rTreeIndexesInfos.get(i), partition);
+                        performOpOnIndex(rTreeIndexesDataflowHelperFactories.get(i), ctx, rTreeIndexesInfos.get(i),
+                                partition);
                     }
                 } catch (Exception e) {
-                    // This should never happen <unless there is a hardware failure or something serious>
                     throw new HyracksDataException(e);
                 }
             }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
similarity index 98%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
index e6e7540..26c9c7d 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.external;
+package org.apache.asterix.external.operators;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
similarity index 98%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 97b3d6a..0513f9c 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.external;
+package org.apache.asterix.external.operators;
 
 import java.io.DataOutput;
 import java.io.IOException;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
similarity index 97%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
index 6ff991b..5255257 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.util.List;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
similarity index 98%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
index e89a8db..71ffef8 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.util.List;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
similarity index 97%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
index 9bdfaa6..59ad076 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.io.File;
 import java.util.List;
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
similarity index 97%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
index bc73eba..0d51956 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
@@ -16,15 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.operators;
 
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
 import org.apache.asterix.external.indexing.FilesIndexDescription;
-import org.apache.asterix.external.indexing.dataflow.FileIndexTupleTranslator;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java
similarity index 98%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java
index f56b3ae..1476c14 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
similarity index 98%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
index a9c9ac7..5748a65 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.nio.ByteBuffer;
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalLoopkupOperatorDiscriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
similarity index 71%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalLoopkupOperatorDiscriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
index ca2e7ca..c8881a3 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/ExternalLoopkupOperatorDiscriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
@@ -16,12 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import java.nio.ByteBuffer;
 
-import org.apache.asterix.external.adapter.factory.IControlledAdapterFactory;
-import org.apache.asterix.external.dataset.adapter.IControlledAdapter;
+import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
+import org.apache.asterix.external.dataset.adapter.LookupAdapter;
 import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
 import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -43,12 +43,11 @@
 /*
  * This operator is intended for using record ids to access data in external sources
  */
-public class ExternalLoopkupOperatorDiscriptor extends AbstractTreeIndexOperatorDescriptor {
+public class ExternalLookupOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
     private static final long serialVersionUID = 1L;
-    private final IControlledAdapterFactory adapterFactory;
-    private final INullWriterFactory iNullWriterFactory;
+    private final LookupAdapterFactory<?> adapterFactory;
 
-    public ExternalLoopkupOperatorDiscriptor(IOperatorDescriptorRegistry spec, IControlledAdapterFactory adapterFactory,
+    public ExternalLookupOperatorDescriptor(IOperatorDescriptorRegistry spec, LookupAdapterFactory<?> adapterFactory,
             RecordDescriptor outRecDesc, ExternalBTreeDataflowHelperFactory externalFilesIndexDataFlowHelperFactory,
             boolean propagateInput, IIndexLifecycleManagerProvider lcManagerProvider,
             IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider, int datasetId,
@@ -60,7 +59,6 @@
                 externalFilesIndexDataFlowHelperFactory, null, propagateInput, retainNull, iNullWriterFactory, null,
                 searchOpCallbackFactory, null);
         this.adapterFactory = adapterFactory;
-        this.iNullWriterFactory = iNullWriterFactory;
     }
 
     @Override
@@ -69,47 +67,44 @@
                     throws HyracksDataException {
         // Create a file index accessor to be used for files lookup operations
         // Note that all file index accessors will use partition 0 since we only have 1 files index per NC 
-        final ExternalFileIndexAccessor fileIndexAccessor = new ExternalFileIndexAccessor(
+        final ExternalFileIndexAccessor snapshotAccessor = new ExternalFileIndexAccessor(
                 (ExternalBTreeDataflowHelper) dataflowHelperFactory.createIndexDataflowHelper(this, ctx, partition),
                 this);
         return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
             // The adapter that uses the file index along with the coming tuples to access files in HDFS
-            private final IControlledAdapter adapter = adapterFactory.createAdapter(ctx, fileIndexAccessor,
-                    recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+            private LookupAdapter<?> adapter;
             private boolean indexOpen = false;
-            private boolean writerOpen = false;
 
             @Override
             public void open() throws HyracksDataException {
-                //Open the file index accessor here
-                fileIndexAccessor.openIndex();
-                indexOpen = true;
                 try {
-                    adapter.initialize(ctx, iNullWriterFactory);
+                    adapter = adapterFactory.createAdapter(ctx, partition,
+                            recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), snapshotAccessor, writer);
+                    //Open the file index accessor here
+                    snapshotAccessor.open();
+                    indexOpen = true;
+                    adapter.open();
                 } catch (Throwable th) {
-                    // close the files index
-                    fileIndexAccessor.closeIndex();
                     throw new HyracksDataException(th);
                 }
-                writerOpen = true;
-                writer.open();
             }
 
             @Override
             public void close() throws HyracksDataException {
-                try {
-                    adapter.close(writer);
-                } catch (Throwable th) {
-                    throw new HyracksDataException(th);
-                } finally {
+                HyracksDataException hde = null;
+                if (indexOpen) {
                     try {
-                        if (indexOpen) {
-                            //close the file index
-                            fileIndexAccessor.closeIndex();
-                        }
-                    } finally {
-                        if (writerOpen) {
-                            writer.close();
+                        snapshotAccessor.close();
+                    } catch (Throwable th) {
+                        hde = new HyracksDataException(th);
+                    }
+                    try {
+                        adapter.close();
+                    } catch (Throwable th) {
+                        if (hde == null) {
+                            hde = new HyracksDataException(th);
+                        } else {
+                            hde.addSuppressed(th);
                         }
                     }
                 }
@@ -121,17 +116,13 @@
                     adapter.fail();
                 } catch (Throwable th) {
                     throw new HyracksDataException(th);
-                } finally {
-                    if (writerOpen) {
-                        writer.fail();
-                    }
                 }
             }
 
             @Override
             public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
                 try {
-                    adapter.nextFrame(buffer, writer);
+                    adapter.nextFrame(buffer);
                 } catch (Throwable th) {
                     throw new HyracksDataException(th);
                 }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
similarity index 98%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
index d34c993..306f75a 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.external;
+package org.apache.asterix.external.operators;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
similarity index 98%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 3fb5609..75cc1bf 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.external;
+package org.apache.asterix.external.operators;
 
 import java.io.DataOutput;
 import java.io.IOException;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
similarity index 98%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
index 6f367d2..99c555a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.operators;
+package org.apache.asterix.external.operators;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.ActivityId;
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
similarity index 94%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index 6e4c175..860d35f 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -16,13 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.operators.file;
+package org.apache.asterix.external.parser;
 
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.BitSet;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.builders.AbvsBuilderFactory;
 import org.apache.asterix.builders.IARecordBuilder;
@@ -33,6 +34,12 @@
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.types.AOrderedListType;
@@ -55,11 +62,12 @@
 /**
  * Parser for ADM formatted data.
  */
-public class ADMDataParser extends AbstractDataParser {
+public class ADMDataParser extends AbstractDataParser implements IStreamDataParser, IRecordDataParser<char[]> {
 
-    protected AdmLexer admLexer;
-    protected ARecordType recordType;
-    protected boolean datasetRec;
+    private AdmLexer admLexer;
+    private ARecordType recordType;
+    private boolean datasetRec;
+    private boolean isStreamParser = true;
 
     private int nullableFieldId = 0;
     private ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
@@ -73,6 +81,7 @@
 
     private String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
     private String mismatchErrorMessage2 = " got a value of type ";
+    private Map<String, String> configuration;
 
     static class ParseException extends AsterixException {
         private static final long serialVersionUID = 1L;
@@ -144,14 +153,51 @@
     }
 
     @Override
-    public void initialize(InputStream in, ARecordType recordType, boolean datasetRec) throws AsterixException {
+    public DataSourceType getDataSourceType() {
+        return ExternalDataUtils.isDataSourceStreamProvider(configuration) ? DataSourceType.STREAM
+                : DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType) throws IOException {
         this.recordType = recordType;
-        this.datasetRec = datasetRec;
-        try {
-            admLexer = new AdmLexer(new java.io.InputStreamReader(in));
-        } catch (IOException e) {
-            throw new ParseException(e);
+        this.configuration = configuration;
+        String isDatasetRecordString = configuration.get(ExternalDataConstants.KEY_DATASET_RECORD);
+        if (isDatasetRecordString == null) {
+            this.datasetRec = true;
+        } else {
+            this.datasetRec = Boolean.parseBoolean(isDatasetRecordString);
         }
+        this.isStreamParser = ExternalDataUtils.isDataSourceStreamProvider(configuration);
+        if (!isStreamParser) {
+            this.admLexer = new AdmLexer();
+        }
+    }
+
+    @Override
+    public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws Exception {
+        try {
+            resetPools();
+            admLexer.setBuffer(record.get());
+            parseAdmInstance(recordType, datasetRec, out);
+        } catch (IOException e) {
+            throw new ParseException(e, filename, admLexer.getLine(), admLexer.getColumn());
+        } catch (AdmLexerException e) {
+            throw new AsterixException(e);
+        } catch (ParseException e) {
+            e.setLocation(filename, admLexer.getLine(), admLexer.getColumn());
+            throw e;
+        }
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    public void setInputStream(InputStream in) throws Exception {
+        admLexer = new AdmLexer(new java.io.InputStreamReader(in));
     }
 
     protected boolean parseAdmInstance(IAType objectType, boolean datasetRec, DataOutput out)
@@ -529,12 +575,14 @@
 
         BitSet nulls = null;
         if (datasetRec) {
+
             if (recType != null) {
                 nulls = new BitSet(recType.getFieldNames().length);
                 recBuilder.reset(recType);
             } else {
                 recBuilder.reset(null);
             }
+
         } else if (recType != null) {
             nulls = new BitSet(recType.getFieldNames().length);
             recBuilder.reset(recType);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
similarity index 99%
rename from asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
index 794097f..f5f2793 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.operators.file;
+package org.apache.asterix.external.parser;
 
 import java.io.DataOutput;
 
+import org.apache.asterix.external.api.IDataParser;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABinary;
 import org.apache.asterix.om.base.ABoolean;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
new file mode 100644
index 0000000..146064a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.Map;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.dataflow.data.nontagged.serde.ANullSerializerDeserializer;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.dataflow.std.file.FieldCursorForDelimitedDataParser;
+
+public class DelimitedDataParser extends AbstractDataParser implements IStreamDataParser, IRecordDataParser<char[]> {
+
+    private final IValueParserFactory[] valueParserFactories;
+    private final char fieldDelimiter;
+    private final char quote;
+    private final boolean hasHeader;
+    private ARecordType recordType;
+    private IARecordBuilder recBuilder;
+    private ArrayBackedValueStorage fieldValueBuffer;
+    private DataOutput fieldValueBufferOutput;
+    private IValueParser[] valueParsers;
+    private FieldCursorForDelimitedDataParser cursor;
+    private byte[] fieldTypeTags;
+    private int[] fldIds;
+    private ArrayBackedValueStorage[] nameBuffers;
+    private boolean areAllNullFields;
+    private boolean isStreamParser = true;
+
+    public DelimitedDataParser(IValueParserFactory[] valueParserFactories, char fieldDelimter, char quote,
+            boolean hasHeader) {
+        this.valueParserFactories = valueParserFactories;
+        this.fieldDelimiter = fieldDelimter;
+        this.quote = quote;
+        this.hasHeader = hasHeader;
+    }
+
+    @Override
+    public boolean parse(DataOutput out) throws AsterixException, IOException {
+        while (cursor.nextRecord()) {
+            parseRecord(out);
+            if (!areAllNullFields) {
+                recBuilder.write(out, true);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private void parseRecord(DataOutput out) throws AsterixException, IOException {
+        recBuilder.reset(recordType);
+        recBuilder.init();
+        areAllNullFields = true;
+
+        for (int i = 0; i < valueParsers.length; ++i) {
+            if (!cursor.nextField()) {
+                break;
+            }
+            fieldValueBuffer.reset();
+
+            if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
+                    && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
+                // if the field is empty and the type is optional, insert
+                // NULL. Note that string type can also process empty field as an
+                // empty string
+                if (!NonTaggedFormatUtil.isOptional(recordType.getFieldTypes()[i])) {
+                    throw new AsterixException("At record: " + cursor.recordCount + " - Field " + cursor.fieldCount
+                            + " is not an optional type so it cannot accept null value. ");
+                }
+                fieldValueBufferOutput.writeByte(ATypeTag.NULL.serialize());
+                ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
+            } else {
+                fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
+                // Eliminate doule quotes in the field that we are going to parse
+                if (cursor.isDoubleQuoteIncludedInThisField) {
+                    cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
+                    cursor.fEnd -= cursor.doubleQuoteCount;
+                    cursor.isDoubleQuoteIncludedInThisField = false;
+                }
+                valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart,
+                        fieldValueBufferOutput);
+                areAllNullFields = false;
+            }
+            if (fldIds[i] < 0) {
+                recBuilder.addField(nameBuffers[i], fieldValueBuffer);
+            } else {
+                recBuilder.addField(fldIds[i], fieldValueBuffer);
+            }
+        }
+    }
+
+    protected void fieldNameToBytes(String fieldName, AMutableString str, ArrayBackedValueStorage buffer)
+            throws HyracksDataException {
+        buffer.reset();
+        DataOutput out = buffer.getDataOutput();
+        str.setValue(fieldName);
+        try {
+            stringSerde.serialize(str, out);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return isStreamParser ? DataSourceType.STREAM : DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType) throws HyracksDataException {
+        this.recordType = recordType;
+        valueParsers = new IValueParser[valueParserFactories.length];
+        for (int i = 0; i < valueParserFactories.length; ++i) {
+            valueParsers[i] = valueParserFactories[i].createValueParser();
+        }
+
+        fieldValueBuffer = new ArrayBackedValueStorage();
+        fieldValueBufferOutput = fieldValueBuffer.getDataOutput();
+        recBuilder = new RecordBuilder();
+        recBuilder.reset(recordType);
+        recBuilder.init();
+
+        int n = recordType.getFieldNames().length;
+        fieldTypeTags = new byte[n];
+        for (int i = 0; i < n; i++) {
+            ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+            fieldTypeTags[i] = tag.serialize();
+        }
+
+        fldIds = new int[n];
+        nameBuffers = new ArrayBackedValueStorage[n];
+        AMutableString str = new AMutableString(null);
+        for (int i = 0; i < n; i++) {
+            String name = recordType.getFieldNames()[i];
+            fldIds[i] = recBuilder.getFieldId(name);
+            if (fldIds[i] < 0) {
+                if (!recordType.isOpen()) {
+                    throw new HyracksDataException("Illegal field " + name + " in closed type " + recordType);
+                } else {
+                    nameBuffers[i] = new ArrayBackedValueStorage();
+                    fieldNameToBytes(name, str, nameBuffers[i]);
+                }
+            }
+        }
+        isStreamParser = ExternalDataUtils.isDataSourceStreamProvider(configuration);
+        if (!isStreamParser) {
+            cursor = new FieldCursorForDelimitedDataParser(null, fieldDelimiter, quote);
+        }
+    }
+
+    @Override
+    public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws Exception {
+        cursor.nextRecord(record.get(), record.size());
+        parseRecord(out);
+        if (!areAllNullFields) {
+            recBuilder.write(out, true);
+        }
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    public void setInputStream(InputStream in) throws Exception {
+        cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote);
+        if (in != null && hasHeader) {
+            cursor.nextRecord();
+            while (cursor.nextField());
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HiveObjectParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
similarity index 62%
rename from asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HiveObjectParser.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
index d7fa4f2..fb61339 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/dataflow/HiveObjectParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.indexing.dataflow;
+package org.apache.asterix.external.parser;
 
 import java.io.DataOutput;
 import java.io.IOException;
@@ -24,7 +24,23 @@
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.conf.Configuration;
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
@@ -41,29 +57,15 @@
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
 import org.apache.hadoop.io.Writable;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.AUnorderedListType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.util.string.UTF8StringWriter;
 
 @SuppressWarnings("deprecation")
-public class HiveObjectParser implements IAsterixHDFSRecordParser {
+public class HiveRecordParser implements IRecordDataParser<Writable> {
 
-    private static final String KEY_HIVE_SERDE = "hive-serde";
     private ARecordType aRecord;
     private SerDe hiveSerde;
     private StructObjectInspector oi;
@@ -74,80 +76,137 @@
     private IAType[] fieldTypes;
     private OrderedListBuilder orderedListBuilder;
     private UnorderedListBuilder unorderedListBuilder;
-    private boolean initialized = false;
-    private List<StructField> fieldRefs;
+    private List<? extends StructField> fieldRefs;
     private UTF8StringWriter utf8Writer = new UTF8StringWriter();
 
-    @SuppressWarnings({ "unchecked" })
     @Override
-    public void initialize(ARecordType record, Map<String, String> arguments, Configuration hadoopConfig)
-            throws Exception {
-        if (!initialized) {
-            this.aRecord = record;
-            int n = record.getFieldNames().length;
-            fieldTypes = record.getFieldTypes();
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
 
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType) throws HyracksDataException {
+        try {
+            this.aRecord = recordType;
+            int n = aRecord.getFieldNames().length;
+            fieldTypes = aRecord.getFieldTypes();
+            JobConf hadoopConfiguration = HDFSUtils.configureHDFSJobConf(configuration);
             //create the hive table schema.
             Properties tbl = new Properties();
-            tbl.put(Constants.LIST_COLUMNS, getCommaDelimitedColNames(record));
-            tbl.put(Constants.LIST_COLUMN_TYPES, getColTypes(record));
-            String hiveSerdeClassName = (String) arguments.get(KEY_HIVE_SERDE);
+            tbl.put(Constants.LIST_COLUMNS, getCommaDelimitedColNames(aRecord));
+            tbl.put(Constants.LIST_COLUMN_TYPES, getColTypes(aRecord));
+            String hiveSerdeClassName = configuration.get(ExternalDataConstants.KEY_HIVE_SERDE);
             if (hiveSerdeClassName == null) {
                 throw new IllegalArgumentException("no hive serde provided for hive deserialized records");
             }
             hiveSerde = (SerDe) Class.forName(hiveSerdeClassName).newInstance();
-            hiveSerde.initialize(hadoopConfig, tbl);
+            hiveSerde.initialize(hadoopConfiguration, tbl);
             oi = (StructObjectInspector) hiveSerde.getObjectInspector();
 
             fieldValueBuffer = new ArrayBackedValueStorage();
             recBuilder = new RecordBuilder();
-            recBuilder.reset(record);
+            recBuilder.reset(aRecord);
             recBuilder.init();
             fieldTypeTags = new byte[n];
             for (int i = 0; i < n; i++) {
-                ATypeTag tag = record.getFieldTypes()[i].getTypeTag();
+                ATypeTag tag = aRecord.getFieldTypes()[i].getTypeTag();
                 fieldTypeTags[i] = tag.serialize();
             }
-            fieldRefs = (List<StructField>) oi.getAllStructFieldRefs();
-            initialized = true;
+            fieldRefs = oi.getAllStructFieldRefs();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
         }
     }
 
+    @Override
+    public void parse(IRawRecord<? extends Writable> record, DataOutput out) throws Exception {
+        Writable hiveRawRecord = record.get();
+        Object hiveObject = hiveSerde.deserialize(hiveRawRecord);
+        int n = aRecord.getFieldNames().length;
+        List<Object> attributesValues = oi.getStructFieldsDataAsList(hiveObject);
+        recBuilder.reset(aRecord);
+        recBuilder.init();
+        for (int i = 0; i < n; i++) {
+            final Object value = attributesValues.get(i);
+            final ObjectInspector foi = fieldRefs.get(i).getFieldObjectInspector();
+            fieldValueBuffer.reset();
+            final DataOutput dataOutput = fieldValueBuffer.getDataOutput();
+            dataOutput.writeByte(fieldTypeTags[i]);
+            //get field type
+            parseItem(fieldTypes[i], value, foi, dataOutput, false);
+            recBuilder.addField(i, fieldValueBuffer);
+        }
+        recBuilder.write(out, true);
+
+    }
+
+    private void parseItem(IAType itemType, Object value, ObjectInspector foi, DataOutput dataOutput,
+            boolean primitiveOnly) throws IOException {
+        switch (itemType.getTypeTag()) {
+            case BOOLEAN:
+                parseBoolean(value, (BooleanObjectInspector) foi, dataOutput);
+                break;
+            case TIME:
+                parseTime(value, (TimestampObjectInspector) foi, dataOutput);
+                break;
+            case DATE:
+                parseDate(value, (TimestampObjectInspector) foi, dataOutput);
+                break;
+            case DATETIME:
+                parseDateTime(value, (TimestampObjectInspector) foi, dataOutput);
+                break;
+            case DOUBLE:
+                parseDouble(value, (DoubleObjectInspector) foi, dataOutput);
+                break;
+            case FLOAT:
+                parseFloat(value, (FloatObjectInspector) foi, dataOutput);
+                break;
+            case INT8:
+                parseInt8(value, (ByteObjectInspector) foi, dataOutput);
+                break;
+            case INT16:
+                parseInt16(value, (ShortObjectInspector) foi, dataOutput);
+                break;
+            case INT32:
+                parseInt32(value, (IntObjectInspector) foi, dataOutput);
+                break;
+            case INT64:
+                parseInt64(value, (LongObjectInspector) foi, dataOutput);
+                break;
+            case STRING:
+                parseString(value, (StringObjectInspector) foi, dataOutput);
+                break;
+            case ORDEREDLIST:
+                if (primitiveOnly) {
+                    throw new HyracksDataException("doesn't support hive data with list of non-primitive types");
+                }
+                parseOrderedList((AOrderedListType) itemType, value, (ListObjectInspector) foi);
+                break;
+            case UNORDEREDLIST:
+                if (primitiveOnly) {
+                    throw new HyracksDataException("doesn't support hive data with list of non-primitive types");
+                }
+                parseUnorderedList((AUnorderedListType) itemType, value, (ListObjectInspector) foi);
+                break;
+            default:
+                throw new HyracksDataException("Can't get hive type for field of type " + itemType.getTypeTag());
+        }
+    }
+
+    @Override
+    public Class<? extends Writable> getRecordClass() {
+        return Writable.class;
+    }
+
     private Object getColTypes(ARecordType record) throws Exception {
         int n = record.getFieldTypes().length;
         if (n < 1) {
             throw new HyracksDataException("Failed to get columns of record");
         }
-        ATypeTag tag = null;
-
         //First Column
-        if (record.getFieldTypes()[0].getTypeTag() == ATypeTag.UNION) {
-            if (NonTaggedFormatUtil.isOptional(record.getFieldTypes()[0])) {
-                throw new NotImplementedException("Non-optional UNION type is not supported.");
-            }
-            tag = ((AUnionType) record.getFieldTypes()[0]).getNullableType().getTypeTag();
-        } else {
-            tag = record.getFieldTypes()[0].getTypeTag();
-        }
-        if (tag == null) {
-            throw new NotImplementedException("Failed to get the type information for field " + 0 + ".");
-        }
-        String cols = getHiveTypeString(tag);
-
+        String cols = getHiveTypeString(record.getFieldTypes(), 0);
         for (int i = 1; i < n; i++) {
-            tag = null;
-            if (record.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
-                if (NonTaggedFormatUtil.isOptional(record.getFieldTypes()[i])) {
-                    throw new NotImplementedException("Non-optional UNION type is not supported.");
-                }
-                tag = ((AUnionType) record.getFieldTypes()[i]).getNullableType().getTypeTag();
-            } else {
-                tag = record.getFieldTypes()[i].getTypeTag();
-            }
-            if (tag == null) {
-                throw new NotImplementedException("Failed to get the type information for field " + i + ".");
-            }
-            cols = cols + "," + getHiveTypeString(tag);
+            cols = cols + "," + getHiveTypeString(record.getFieldTypes(), i);
         }
         return cols;
     }
@@ -164,7 +223,18 @@
         return cols;
     }
 
-    private String getHiveTypeString(ATypeTag tag) throws Exception {
+    private String getHiveTypeString(IAType[] types, int i) throws Exception {
+        final IAType type = types[i];
+        ATypeTag tag = type.getTypeTag();
+        if (tag == ATypeTag.UNION) {
+            if (NonTaggedFormatUtil.isOptional(type)) {
+                throw new NotImplementedException("Non-optional UNION type is not supported.");
+            }
+            tag = ((AUnionType) type).getNullableType().getTypeTag();
+        }
+        if (tag == null) {
+            throw new NotImplementedException("Failed to get the type information for field " + i + ".");
+        }
         switch (tag) {
             case BOOLEAN:
                 return Constants.BOOLEAN_TYPE_NAME;
@@ -197,74 +267,6 @@
         }
     }
 
-    @Override
-    public void parse(Object object, DataOutput output) throws Exception {
-        if (object == null) {
-            throw new HyracksDataException("Hive parser can't parse null objects");
-        }
-        Object hiveObject = hiveSerde.deserialize((Writable) object);
-        int n = aRecord.getFieldNames().length;
-        List<Object> attributesValues = oi.getStructFieldsDataAsList(hiveObject);
-        recBuilder.reset(aRecord);
-        recBuilder.init();
-        for (int i = 0; i < n; i++) {
-            fieldValueBuffer.reset();
-            fieldValueBuffer.getDataOutput().writeByte(fieldTypeTags[i]);
-            ObjectInspector foi = fieldRefs.get(i).getFieldObjectInspector();
-            //get field type
-            switch (fieldTypes[i].getTypeTag()) {
-                case BOOLEAN:
-                    parseBoolean(attributesValues.get(i), (BooleanObjectInspector) foi,
-                            fieldValueBuffer.getDataOutput());
-                    break;
-                case TIME:
-                    parseTime(attributesValues.get(i), (TimestampObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case DATE:
-                    parseDate(attributesValues.get(i), (TimestampObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case DATETIME:
-                    parseDateTime(attributesValues.get(i), (TimestampObjectInspector) foi,
-                            fieldValueBuffer.getDataOutput());
-                    break;
-                case DOUBLE:
-                    parseDouble(attributesValues.get(i), (DoubleObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case FLOAT:
-                    parseFloat(attributesValues.get(i), (FloatObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case INT8:
-                    parseInt8(attributesValues.get(i), (ByteObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case INT16:
-                    parseInt16(attributesValues.get(i), (ShortObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case INT32:
-                    parseInt32(attributesValues.get(i), (IntObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case INT64:
-                    parseInt64(attributesValues.get(i), (LongObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case STRING:
-                    parseString(attributesValues.get(i), (StringObjectInspector) foi, fieldValueBuffer.getDataOutput());
-                    break;
-                case ORDEREDLIST:
-                    parseOrderedList((AOrderedListType) fieldTypes[i], attributesValues.get(i),
-                            (ListObjectInspector) foi);
-                    break;
-                case UNORDEREDLIST:
-                    parseUnorderedList((AUnorderedListType) fieldTypes[i], attributesValues.get(i),
-                            (ListObjectInspector) foi);
-                    break;
-                default:
-                    throw new HyracksDataException("Can't get hive type for field of type "
-                            + fieldTypes[i].getTypeTag());
-            }
-            recBuilder.addField(i, fieldValueBuffer);
-        }
-        recBuilder.write(output, true);
-    }
-
     private void parseInt64(Object obj, LongObjectInspector foi, DataOutput dataOutput) throws IOException {
         dataOutput.writeLong(foi.get(obj));
     }
@@ -332,15 +334,13 @@
             if (element == null) {
                 throw new HyracksDataException("can't parse hive list with null values");
             }
-
-            parseHiveListItem(element, eoi, listItemBuffer, itemType);
+            parseItem(itemType, element, eoi, listItemBuffer.getDataOutput(), true);
             orderedListBuilder.addItem(listItemBuffer);
         }
         orderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
     }
 
-    private void parseUnorderedList(AUnorderedListType uoltype, Object obj, ListObjectInspector oi) throws IOException,
-            AsterixException {
+    private void parseUnorderedList(AUnorderedListType uoltype, Object obj, ListObjectInspector oi) throws IOException {
         UnorderedListBuilder unorderedListBuilder = getUnorderedListBuilder();
         IAType itemType = null;
         if (uoltype != null)
@@ -356,55 +356,14 @@
                 throw new HyracksDataException("can't parse hive list with null values");
             }
             listItemBuffer.reset();
-            listItemBuffer.getDataOutput().writeByte(tagByte);
-            parseHiveListItem(element, eoi, listItemBuffer, itemType);
+            final DataOutput dataOutput = listItemBuffer.getDataOutput();
+            dataOutput.writeByte(tagByte);
+            parseItem(itemType, element, eoi, dataOutput, true);
             unorderedListBuilder.addItem(listItemBuffer);
         }
         unorderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
     }
 
-    private void parseHiveListItem(Object obj, ObjectInspector eoi, ArrayBackedValueStorage fieldValueBuffer,
-            IAType itemType) throws IOException {
-        //get field type
-        switch (itemType.getTypeTag()) {
-            case BOOLEAN:
-                parseBoolean(obj, (BooleanObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case TIME:
-                parseTime(obj, (TimestampObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case DATE:
-                parseDate(obj, (TimestampObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case DATETIME:
-                parseDateTime(obj, (TimestampObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case DOUBLE:
-                parseDouble(obj, (DoubleObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case FLOAT:
-                parseFloat(obj, (FloatObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case INT8:
-                parseInt8(obj, (ByteObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case INT16:
-                parseInt16(obj, (ShortObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case INT32:
-                parseInt32(obj, (IntObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case INT64:
-                parseInt64(obj, (LongObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            case STRING:
-                parseString(obj, (StringObjectInspector) eoi, fieldValueBuffer.getDataOutput());
-                break;
-            default:
-                throw new HyracksDataException("doesn't support hive data with list of non-primitive types");
-        }
-    }
-
     private OrderedListBuilder getOrderedListBuilder() {
         if (orderedListBuilder != null)
             return orderedListBuilder;
@@ -423,4 +382,4 @@
         }
     }
 
-}
\ No newline at end of file
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java
new file mode 100644
index 0000000..4d93dc5
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.external.api.IDataParser;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.om.base.AMutableRecord;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+
+public class RSSParser implements IRecordDataParser<SyndEntryImpl> {
+    private long id = 0;
+    private String idPrefix;
+    private AMutableString[] mutableFields;
+    private String[] tupleFieldValues;
+    private AMutableRecord mutableRecord;
+    private RecordBuilder recordBuilder = new RecordBuilder();
+    private int numFields;
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType)
+            throws HyracksDataException, IOException {
+        mutableFields = new AMutableString[] { new AMutableString(null), new AMutableString(null),
+                new AMutableString(null), new AMutableString(null) };
+        mutableRecord = new AMutableRecord(recordType, mutableFields);
+        tupleFieldValues = new String[recordType.getFieldNames().length];
+        numFields = recordType.getFieldNames().length;
+    }
+
+    @Override
+    public void parse(IRawRecord<? extends SyndEntryImpl> record, DataOutput out) throws Exception {
+        SyndEntryImpl entry = record.get();
+        tupleFieldValues[0] = idPrefix + ":" + id;
+        tupleFieldValues[1] = entry.getTitle();
+        tupleFieldValues[2] = entry.getDescription().getValue();
+        tupleFieldValues[3] = entry.getLink();
+        for (int i = 0; i < numFields; i++) {
+            mutableFields[i].setValue(tupleFieldValues[i]);
+            mutableRecord.setValueAtPos(i, mutableFields[i]);
+        }
+        recordBuilder.reset(mutableRecord.getType());
+        recordBuilder.init();
+        IDataParser.writeRecord(mutableRecord, out, recordBuilder);
+        id++;
+    }
+
+    @Override
+    public Class<? extends SyndEntryImpl> getRecordClass() {
+        return SyndEntryImpl.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java
new file mode 100644
index 0000000..b9cd60b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.external.api.IDataParser;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.library.java.JObjectUtil;
+import org.apache.asterix.external.util.Datatypes.Tweet;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableRecord;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import twitter4j.Status;
+import twitter4j.User;
+
+public class TweetParser implements IRecordDataParser<Status> {
+
+    private IAObject[] mutableTweetFields;
+    private IAObject[] mutableUserFields;
+    private AMutableRecord mutableRecord;
+    private AMutableRecord mutableUser;
+    private final Map<String, Integer> userFieldNameMap = new HashMap<>();
+    private final Map<String, Integer> tweetFieldNameMap = new HashMap<>();
+    private RecordBuilder recordBuilder = new RecordBuilder();
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType)
+            throws HyracksDataException, IOException {
+        initFieldNames(recordType);
+        mutableUserFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableInt32(0),
+                new AMutableInt32(0), new AMutableString(null), new AMutableInt32(0) };
+        mutableUser = new AMutableRecord((ARecordType) recordType.getFieldTypes()[tweetFieldNameMap.get(Tweet.USER)],
+                mutableUserFields);
+
+        mutableTweetFields = new IAObject[] { new AMutableString(null), mutableUser, new AMutableDouble(0),
+                new AMutableDouble(0), new AMutableString(null), new AMutableString(null) };
+        mutableRecord = new AMutableRecord(recordType, mutableTweetFields);
+
+    }
+
+    // Initialize the hashmap values for the field names and positions
+    private void initFieldNames(ARecordType recordType) {
+        String tweetFields[] = recordType.getFieldNames();
+        for (int i = 0; i < tweetFields.length; i++) {
+            tweetFieldNameMap.put(tweetFields[i], i);
+            if (tweetFields[i].equals(Tweet.USER)) {
+                IAType fieldType = recordType.getFieldTypes()[i];
+                if (fieldType.getTypeTag() == ATypeTag.RECORD) {
+                    String userFields[] = ((ARecordType) fieldType).getFieldNames();
+                    for (int j = 0; j < userFields.length; j++) {
+                        userFieldNameMap.put(userFields[j], j);
+                    }
+                }
+
+            }
+        }
+    }
+
+    @Override
+    public void parse(IRawRecord<? extends Status> record, DataOutput out) throws Exception {
+        Status tweet = record.get();
+        User user = tweet.getUser();
+        // Tweet user data
+        ((AMutableString) mutableUserFields[userFieldNameMap.get(Tweet.SCREEN_NAME)])
+                .setValue(JObjectUtil.getNormalizedString(user.getScreenName()));
+        ((AMutableString) mutableUserFields[userFieldNameMap.get(Tweet.LANGUAGE)])
+                .setValue(JObjectUtil.getNormalizedString(user.getLang()));
+        ((AMutableInt32) mutableUserFields[userFieldNameMap.get(Tweet.FRIENDS_COUNT)]).setValue(user.getFriendsCount());
+        ((AMutableInt32) mutableUserFields[userFieldNameMap.get(Tweet.STATUS_COUNT)]).setValue(user.getStatusesCount());
+        ((AMutableString) mutableUserFields[userFieldNameMap.get(Tweet.NAME)])
+                .setValue(JObjectUtil.getNormalizedString(user.getName()));
+        ((AMutableInt32) mutableUserFields[userFieldNameMap.get(Tweet.FOLLOWERS_COUNT)])
+                .setValue(user.getFollowersCount());
+
+        // Tweet data
+        ((AMutableString) mutableTweetFields[tweetFieldNameMap.get(Tweet.ID)]).setValue(String.valueOf(tweet.getId()));
+
+        int userPos = tweetFieldNameMap.get(Tweet.USER);
+        for (int i = 0; i < mutableUserFields.length; i++) {
+            ((AMutableRecord) mutableTweetFields[userPos]).setValueAtPos(i, mutableUserFields[i]);
+        }
+        if (tweet.getGeoLocation() != null) {
+            ((AMutableDouble) mutableTweetFields[tweetFieldNameMap.get(Tweet.LATITUDE)])
+                    .setValue(tweet.getGeoLocation().getLatitude());
+            ((AMutableDouble) mutableTweetFields[tweetFieldNameMap.get(Tweet.LONGITUDE)])
+                    .setValue(tweet.getGeoLocation().getLongitude());
+        } else {
+            ((AMutableDouble) mutableTweetFields[tweetFieldNameMap.get(Tweet.LATITUDE)]).setValue(0);
+            ((AMutableDouble) mutableTweetFields[tweetFieldNameMap.get(Tweet.LONGITUDE)]).setValue(0);
+        }
+        ((AMutableString) mutableTweetFields[tweetFieldNameMap.get(Tweet.CREATED_AT)])
+                .setValue(JObjectUtil.getNormalizedString(tweet.getCreatedAt().toString()));
+        ((AMutableString) mutableTweetFields[tweetFieldNameMap.get(Tweet.MESSAGE)])
+                .setValue(JObjectUtil.getNormalizedString(tweet.getText()));
+
+        for (int i = 0; i < mutableTweetFields.length; i++) {
+            mutableRecord.setValueAtPos(i, mutableTweetFields[i]);
+        }
+        recordBuilder.reset(mutableRecord.getType());
+        recordBuilder.init();
+        IDataParser.writeRecord(mutableRecord, out, recordBuilder);
+    }
+
+    @Override
+    public Class<? extends Status> getRecordClass() {
+        return Status.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/ADMDataParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/ADMDataParserFactory.java
new file mode 100644
index 0000000..4634278
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/ADMDataParserFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.parser.ADMDataParser;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ADMDataParserFactory extends AbstractRecordStreamParserFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordDataParser<char[]> createRecordParser(IHyracksTaskContext ctx) throws HyracksDataException {
+        return createParser();
+    }
+
+    private ADMDataParser createParser() throws HyracksDataException {
+        try {
+            ADMDataParser parser = new ADMDataParser();
+            parser.configure(configuration, recordType);
+            return parser;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    public IStreamDataParser createInputStreamParser(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        return createParser();
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/AbstractRecordStreamParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/AbstractRecordStreamParserFactory.java
new file mode 100644
index 0000000..43af455
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/AbstractRecordStreamParserFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.api.IStreamDataParserFactory;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.om.types.ARecordType;
+
+public abstract class AbstractRecordStreamParserFactory<T>
+        implements IStreamDataParserFactory, IRecordDataParserFactory<T> {
+
+    private static final long serialVersionUID = 1L;
+    protected ARecordType recordType;
+    protected Map<String, String> configuration;
+
+    @Override
+    public DataSourceType getDataSourceType() throws AsterixException {
+        return ExternalDataUtils.getDataSourceType(configuration);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) {
+        this.recordType = recordType;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java
new file mode 100644
index 0000000..fa63d45
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.parser.DelimitedDataParser;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+
+public class DelimitedDataParserFactory extends AbstractRecordStreamParserFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordDataParser<char[]> createRecordParser(IHyracksTaskContext ctx)
+            throws HyracksDataException, AsterixException {
+        return createParser();
+    }
+
+    private DelimitedDataParser createParser() throws HyracksDataException, AsterixException {
+        IValueParserFactory[] valueParserFactories = ExternalDataUtils.getValueParserFactories(recordType);
+        Character delimiter = DelimitedDataParserFactory.getDelimiter(configuration);
+        char quote = DelimitedDataParserFactory.getQuote(configuration, delimiter);
+        boolean hasHeader = ExternalDataUtils.hasHeader(configuration);
+        DelimitedDataParser parser = new DelimitedDataParser(valueParserFactories, delimiter, quote, hasHeader);
+        parser.configure(configuration, recordType);
+        return parser;
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    public IStreamDataParser createInputStreamParser(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException, AsterixException {
+        return createParser();
+    }
+
+    // Get a delimiter from the given configuration
+    public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
+        String delimiterValue = configuration.get(ExternalDataConstants.KEY_DELIMITER);
+        if (delimiterValue == null) {
+            delimiterValue = ExternalDataConstants.DEFAULT_DELIMITER;
+        } else if (delimiterValue.length() != 1) {
+            throw new AsterixException(
+                    "'" + delimiterValue + "' is not a valid delimiter. The length of a delimiter should be 1.");
+        }
+        return delimiterValue.charAt(0);
+    }
+
+    // Get a quote from the given configuration when the delimiter is given
+    // Need to pass delimiter to check whether they share the same character
+    public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
+        String quoteValue = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        if (quoteValue == null) {
+            quoteValue = ExternalDataConstants.DEFAULT_QUOTE;
+        } else if (quoteValue.length() != 1) {
+            throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
+        }
+
+        // Since delimiter (char type value) can't be null,
+        // we only check whether delimiter and quote use the same character
+        if (quoteValue.charAt(0) == delimiter) {
+            throw new AsterixException(
+                    "Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter + "'. ");
+        }
+
+        return quoteValue.charAt(0);
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
new file mode 100644
index 0000000..f07ba4c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.parser.HiveRecordParser;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hadoop.io.Writable;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class HiveDataParserFactory implements IRecordDataParserFactory<Writable> {
+
+    private static final long serialVersionUID = 1L;
+    private Map<String, String> configuration;
+    private ARecordType recordType;
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) {
+        this.recordType = recordType;
+    }
+
+    @Override
+    public IRecordDataParser<Writable> createRecordParser(IHyracksTaskContext ctx)
+            throws HyracksDataException, AsterixException {
+        HiveRecordParser hiveParser = new HiveRecordParser();
+        hiveParser.configure(configuration, recordType);
+        return hiveParser;
+    }
+
+    @Override
+    public Class<? extends Writable> getRecordClass() {
+        return Writable.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RSSParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RSSParserFactory.java
new file mode 100644
index 0000000..fecb0de
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RSSParserFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.parser.RSSParser;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+
+public class RSSParserFactory implements IRecordDataParserFactory<SyndEntryImpl> {
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+    private Map<String, String> configuration;
+
+    @Override
+    public DataSourceType getDataSourceType() throws AsterixException {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) {
+        this.recordType = recordType;
+    }
+
+    @Override
+    public IRecordDataParser<SyndEntryImpl> createRecordParser(IHyracksTaskContext ctx)
+            throws AsterixException, IOException {
+        RSSParser dataParser = new RSSParser();
+        dataParser.configure(configuration, recordType);
+        return dataParser;
+    }
+
+    @Override
+    public Class<? extends SyndEntryImpl> getRecordClass() {
+        return SyndEntryImpl.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java
new file mode 100644
index 0000000..0f3b309
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser.factory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.parser.TweetParser;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import twitter4j.Status;
+
+public class TweetParserFactory implements IRecordDataParserFactory<Status> {
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+    private Map<String, String> configuration;
+
+    @Override
+    public DataSourceType getDataSourceType() throws AsterixException {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) {
+        this.recordType = recordType;
+    }
+
+    @Override
+    public IRecordDataParser<Status> createRecordParser(IHyracksTaskContext ctx) throws AsterixException, IOException {
+        TweetParser dataParser = new TweetParser();
+        dataParser.configure(configuration, recordType);
+        return dataParser;
+    }
+
+    @Override
+    public Class<? extends Status> getRecordClass() {
+        return Status.class;
+    }
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
new file mode 100644
index 0000000..649ca43
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.provider;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
+import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.IIndexingAdapterFactory;
+import org.apache.asterix.external.dataset.adapter.GenericAdapter;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.library.ExternalLibraryManager;
+import org.apache.asterix.external.runtime.GenericSocketFeedAdapter;
+import org.apache.asterix.external.runtime.GenericSocketFeedAdapterFactory;
+import org.apache.asterix.external.runtime.SocketClientAdapter;
+import org.apache.asterix.external.runtime.SocketClientAdapterFactory;
+import org.apache.asterix.external.util.ExternalDataCompatibilityUtils;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
+
+public class AdapterFactoryProvider {
+
+    public static final Map<String, Class<? extends IAdapterFactory>> adapterFactories = initializeAdapterFactoryMapping();
+
+    private static Map<String, Class<? extends IAdapterFactory>> initializeAdapterFactoryMapping() {
+        Map<String, Class<? extends IAdapterFactory>> adapterFactories = new HashMap<String, Class<? extends IAdapterFactory>>();
+        // Class names
+        adapterFactories.put(GenericAdapter.class.getName(), GenericAdapterFactory.class);
+        adapterFactories.put(GenericSocketFeedAdapter.class.getName(), GenericSocketFeedAdapterFactory.class);
+        adapterFactories.put(SocketClientAdapter.class.getName(), SocketClientAdapterFactory.class);
+
+        // Aliases
+        adapterFactories.put(ExternalDataConstants.ALIAS_GENERIC_ADAPTER, GenericAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ALIAS_HDFS_ADAPTER, GenericAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ALIAS_LOCALFS_ADAPTER, GenericAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ALIAS_SOCKET_ADAPTER, GenericSocketFeedAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ALIAS_SOCKET_CLIENT_ADAPTER, SocketClientAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ALIAS_FILE_FEED_ADAPTER, GenericAdapterFactory.class);
+
+        // Compatability
+        adapterFactories.put(ExternalDataConstants.ADAPTER_HDFS_CLASSNAME, GenericAdapterFactory.class);
+        adapterFactories.put(ExternalDataConstants.ADAPTER_LOCALFS_CLASSNAME, GenericAdapterFactory.class);
+        return adapterFactories;
+    }
+
+    public static IAdapterFactory getAdapterFactory(String adapterClassname, Map<String, String> configuration,
+            ARecordType itemType) throws Exception {
+        ExternalDataCompatibilityUtils.addCompatabilityParameters(adapterClassname, itemType, configuration);
+        if (!adapterFactories.containsKey(adapterClassname)) {
+            throw new AsterixException("Unknown adapter: " + adapterClassname);
+        }
+        IAdapterFactory adapterFactory = adapterFactories.get(adapterClassname).newInstance();
+        adapterFactory.configure(configuration, itemType);
+        return adapterFactory;
+    }
+
+    public static IIndexingAdapterFactory getAdapterFactory(String adapterClassname, Map<String, String> configuration,
+            ARecordType itemType, List<ExternalFile> snapshot, boolean indexingOp)
+                    throws AsterixException, InstantiationException, IllegalAccessException {
+        ExternalDataCompatibilityUtils.addCompatabilityParameters(adapterClassname, itemType, configuration);
+        if (!adapterFactories.containsKey(adapterClassname)) {
+            throw new AsterixException("Unknown adapter");
+        }
+        try {
+            IIndexingAdapterFactory adapterFactory = (IIndexingAdapterFactory) adapterFactories.get(adapterClassname)
+                    .newInstance();
+            adapterFactory.setSnapshot(snapshot, indexingOp);
+            adapterFactory.configure(configuration, itemType);
+            return adapterFactory;
+        } catch (Exception e) {
+            throw new AsterixException("Failed to create indexing adapter factory.", e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void addNewAdapter(String dataverseName, String adapterClassName, String adapterAlias,
+            String adapterFactoryClassName, String libraryName) throws ClassNotFoundException {
+        ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(dataverseName, libraryName);
+        Class<? extends IAdapterFactory> adapterFactoryClass = (Class<? extends IAdapterFactory>) classLoader
+                .loadClass(adapterFactoryClassName);
+        adapterFactories.put(adapterClassName, adapterFactoryClass);
+        adapterFactories.put(adapterAlias, adapterFactoryClass);
+    }
+
+    public static LookupAdapterFactory<?> getAdapterFactory(Map<String, String> configuration, ARecordType recordType,
+            int[] ridFields, boolean retainInput, boolean retainNull, INullWriterFactory iNullWriterFactory)
+                    throws Exception {
+        LookupAdapterFactory<?> adapterFactory = new LookupAdapterFactory<>(recordType, ridFields, retainInput,
+                retainNull, iNullWriterFactory);
+        adapterFactory.configure(configuration);
+        return adapterFactory;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java
new file mode 100644
index 0000000..68a3942
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.provider;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.api.IExternalDataSourceFactory;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.api.IStreamDataParserFactory;
+import org.apache.asterix.external.dataflow.IndexingDataFlowController;
+import org.apache.asterix.external.dataflow.RecordDataFlowController;
+import org.apache.asterix.external.dataflow.StreamDataFlowController;
+import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class DataflowControllerProvider {
+
+    /**
+     * Order of calls:
+     * 1. Constructor()
+     * 2. configure(configuration,ctx)
+     * 3. setTupleForwarder(forwarder)
+     * 4. if record flow controller
+     * |-a. Set record reader
+     * |-b. Set record parser
+     * else
+     * |-a. Set stream parser
+     * 5. start(writer)
+     */
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    public static IDataFlowController getDataflowController(ARecordType recordType, IHyracksTaskContext ctx,
+            int partition, IExternalDataSourceFactory dataSourceFactory, IDataParserFactory dataParserFactory,
+            Map<String, String> configuration, boolean indexingOp) throws Exception {
+        switch (dataSourceFactory.getDataSourceType()) {
+            case RECORDS:
+                RecordDataFlowController recordDataFlowController;
+                if (indexingOp) {
+                    recordDataFlowController = new IndexingDataFlowController();
+                } else {
+                    recordDataFlowController = new RecordDataFlowController();
+                }
+                recordDataFlowController.configure(configuration, ctx);
+                recordDataFlowController.setTupleForwarder(DataflowUtils.getTupleForwarder(configuration));
+                IRecordReaderFactory<?> recordReaderFactory = (IRecordReaderFactory<?>) dataSourceFactory;
+                IRecordReader<?> recordReader = recordReaderFactory.createRecordReader(ctx, partition);
+                IRecordDataParserFactory<?> recordParserFactory = (IRecordDataParserFactory<?>) dataParserFactory;
+                IRecordDataParser<?> dataParser = recordParserFactory.createRecordParser(ctx);
+                dataParser.configure(configuration, recordType);
+                recordDataFlowController.setRecordReader(recordReader);
+                recordDataFlowController.setRecordParser(dataParser);
+                return recordDataFlowController;
+            case STREAM:
+                StreamDataFlowController streamDataFlowController = new StreamDataFlowController();
+                streamDataFlowController.configure(configuration, ctx);
+                streamDataFlowController.setTupleForwarder(DataflowUtils.getTupleForwarder(configuration));
+                IInputStreamProviderFactory streamProviderFactory = (IInputStreamProviderFactory) dataSourceFactory;
+                IInputStreamProvider streamProvider = streamProviderFactory.createInputStreamProvider(ctx, partition);
+                IStreamDataParserFactory streamParserFactory = (IStreamDataParserFactory) dataParserFactory;
+                streamParserFactory.configure(configuration);
+                IStreamDataParser streamParser = streamParserFactory.createInputStreamParser(ctx, partition);
+                streamParser.configure(configuration, recordType);
+                streamParser.setInputStream(streamProvider.getInputStream());
+                streamDataFlowController.setStreamParser(streamParser);
+                return streamDataFlowController;
+            default:
+                throw new AsterixException("Unknown data source type: " + dataSourceFactory.getDataSourceType());
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
new file mode 100644
index 0000000..c69e12c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.provider;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.factory.LineRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.factory.SemiStructuredRecordReaderFactory;
+import org.apache.asterix.external.input.stream.factory.LocalFSInputStreamProviderFactory;
+import org.apache.asterix.external.input.stream.factory.SocketInputStreamProviderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+
+public class DatasourceFactoryProvider {
+
+    public static IExternalDataSourceFactory getExternalDataSourceFactory(Map<String, String> configuration)
+            throws Exception {
+        switch (ExternalDataUtils.getDataSourceType(configuration)) {
+            case RECORDS:
+                return DatasourceFactoryProvider.getRecordReaderFactory(configuration);
+            case STREAM:
+                return DatasourceFactoryProvider
+                        .getInputStreamFactory(configuration.get(ExternalDataConstants.KEY_STREAM), configuration);
+        }
+        return null;
+    }
+
+    public static IInputStreamProviderFactory getInputStreamFactory(String stream, Map<String, String> configuration)
+            throws Exception {
+        IInputStreamProviderFactory streamFactory;
+        if (ExternalDataUtils.isExternal(stream)) {
+            String dataverse = ExternalDataUtils.getDataverse(configuration);
+            streamFactory = ExternalDataUtils.createExternalInputStreamFactory(dataverse, stream);
+        } else {
+            switch (stream) {
+                case ExternalDataConstants.STREAM_HDFS:
+                    streamFactory = new HDFSDataSourceFactory();
+                    break;
+                case ExternalDataConstants.STREAM_LOCAL_FILESYSTEM:
+                    streamFactory = new LocalFSInputStreamProviderFactory();
+                    break;
+                case ExternalDataConstants.STREAM_SOCKET:
+                    streamFactory = new SocketInputStreamProviderFactory();
+                    break;
+                default:
+                    throw new AsterixException("unknown input stream factory");
+            }
+        }
+        return streamFactory;
+    }
+
+    public static IRecordReaderFactory<?> getRecordReaderFactory(Map<String, String> configuration) throws Exception {
+        String reader = configuration.get(ExternalDataConstants.KEY_READER);
+        IRecordReaderFactory<?> readerFactory;
+        if (ExternalDataUtils.isExternal(reader)) {
+            String dataverse = ExternalDataUtils.getDataverse(configuration);
+            readerFactory = ExternalDataUtils.createExternalRecordReaderFactory(dataverse, reader);
+        } else {
+            switch (reader) {
+                case ExternalDataConstants.READER_HDFS:
+                    readerFactory = new HDFSDataSourceFactory();
+                    break;
+                case ExternalDataConstants.READER_ADM:
+                case ExternalDataConstants.READER_SEMISTRUCTURED:
+                    readerFactory = new SemiStructuredRecordReaderFactory()
+                            .setInputStreamFactoryProvider(DatasourceFactoryProvider.getInputStreamFactory(
+                                    ExternalDataUtils.getRecordReaderStreamName(configuration), configuration));
+                    break;
+                case ExternalDataConstants.READER_DELIMITED:
+                    readerFactory = new LineRecordReaderFactory()
+                            .setInputStreamFactoryProvider(DatasourceFactoryProvider.getInputStreamFactory(
+                                    ExternalDataUtils.getRecordReaderStreamName(configuration), configuration));;
+                    break;
+                default:
+                    throw new AsterixException("unknown input stream factory");
+            }
+        }
+        return readerFactory;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
new file mode 100644
index 0000000..3c090a6
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.provider;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.indexing.FileOffsetIndexer;
+import org.apache.asterix.external.indexing.RecordColumnarIndexer;
+import org.apache.asterix.external.util.ExternalDataConstants;
+
+public class ExternalIndexerProvider {
+
+    public static IExternalIndexer getIndexer(Map<String, String> configuration) throws AsterixException {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        if (inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.INPUT_FORMAT_TEXT)
+                || inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
+                || inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.INPUT_FORMAT_SEQUENCE)
+                || inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)) {
+            return new FileOffsetIndexer();
+        } else if (inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.INPUT_FORMAT_RC)
+                || inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
+            return new RecordColumnarIndexer();
+        } else {
+            throw new AsterixException("Unable to create indexer for data with format: " + inputFormatParameter);
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
new file mode 100644
index 0000000..f5a0512
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.provider;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.parser.factory.ADMDataParserFactory;
+import org.apache.asterix.external.parser.factory.DelimitedDataParserFactory;
+import org.apache.asterix.external.parser.factory.HiveDataParserFactory;
+import org.apache.asterix.external.parser.factory.RSSParserFactory;
+import org.apache.asterix.external.parser.factory.TweetParserFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+
+public class ParserFactoryProvider {
+    public static IDataParserFactory getDataParserFactory(Map<String, String> configuration)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException, AsterixException {
+        IDataParserFactory parserFactory = null;
+        String parserFactoryName = configuration.get(ExternalDataConstants.KEY_DATA_PARSER);
+        if (parserFactoryName != null && ExternalDataUtils.isExternal(parserFactoryName)) {
+            return ExternalDataUtils.createExternalParserFactory(ExternalDataUtils.getDataverse(configuration),
+                    parserFactoryName);
+        } else {
+            parserFactory = ParserFactoryProvider.getParserFactory(configuration);
+        }
+        return parserFactory;
+    }
+
+    private static IDataParserFactory getParserFactory(Map<String, String> configuration) throws AsterixException {
+        String recordFormat = ExternalDataUtils.getRecordFormat(configuration);
+        switch (recordFormat) {
+            case ExternalDataConstants.FORMAT_ADM:
+            case ExternalDataConstants.FORMAT_JSON:
+                return new ADMDataParserFactory();
+            case ExternalDataConstants.FORMAT_DELIMITED_TEXT:
+                return new DelimitedDataParserFactory();
+            case ExternalDataConstants.FORMAT_HIVE:
+                return new HiveDataParserFactory();
+            case ExternalDataConstants.FORMAT_TWEET:
+                return new TweetParserFactory();
+            case ExternalDataConstants.FORMAT_RSS:
+                return new RSSParserFactory();
+            default:
+                throw new AsterixException("Unknown data format");
+        }
+    }
+}
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/DataGenerator.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/DataGenerator.java
similarity index 99%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/DataGenerator.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/DataGenerator.java
index 43db6c4..cbf488c 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/DataGenerator.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/DataGenerator.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.nio.CharBuffer;
 import java.util.ArrayList;
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapter.java
similarity index 92%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapter.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapter.java
index 7dd5130..dcf3b51 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapter.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -25,14 +25,13 @@
 import java.util.logging.Level;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
 import org.apache.asterix.external.dataset.adapter.StreamBasedAdapter;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public class GenericSocketFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+public class GenericSocketFeedAdapter extends StreamBasedAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -102,16 +101,13 @@
     }
 
     @Override
-    public void stop() throws Exception {
+    public boolean stop() throws Exception {
         socketFeedServer.stop();
-    }
-
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PUSH;
+        return true;
     }
 
     @Override
-    public boolean handleException(Exception e) {
+    public boolean handleException(Throwable e) {
         try {
             this.socketFeedServer = new SocketFeedServer((ARecordType) sourceDatatype, port);
             return true;
@@ -119,5 +115,4 @@
             return false;
         }
     }
-
 }
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapterFactory.java
similarity index 74%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapterFactory.java
index 5d28f3d..e19d757 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/GenericSocketFeedAdapterFactory.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -25,28 +25,23 @@
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.commons.lang3.StringUtils;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
-import org.apache.asterix.external.adapter.factory.StreamBasedAdapterFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.util.AsterixRuntimeUtil;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
 /**
  * Factory class for creating @see{GenericSocketFeedAdapter} The
  * adapter listens at a port for receiving data (from external world).
  * Data received is transformed into Asterix Data Format (ADM).
  */
-public class GenericSocketFeedAdapterFactory extends StreamBasedAdapterFactory implements IFeedAdapterFactory {
+public class GenericSocketFeedAdapterFactory implements IAdapterFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -56,6 +51,10 @@
 
     private Mode mode = Mode.IP;
 
+    private Map<String, String> configuration;
+
+    private ITupleParserFactory parserFactory;
+
     public static final String KEY_SOCKETS = "sockets";
 
     public static final String KEY_MODE = "address-type";
@@ -66,15 +65,10 @@
     }
 
     @Override
-    public String getName() {
+    public String getAlias() {
         return "socket_adapter";
     }
 
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
     public List<Pair<String, Integer>> getSockets() {
         return sockets;
     }
@@ -84,7 +78,12 @@
         this.configuration = configuration;
         this.configureSockets(configuration);
         this.configureFormat(outputType);
-        this.outputType = (ARecordType) outputType;
+        this.outputType = outputType;
+    }
+
+    private void configureFormat(ARecordType outputType2) {
+        // TODO Auto-generated method stub
+
     }
 
     @Override
@@ -97,7 +96,7 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+    public IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
         Pair<String, Integer> socket = sockets.get(partition);
         return new GenericSocketFeedAdapter(parserFactory, outputType, socket.second, ctx, partition);
     }
@@ -137,9 +136,9 @@
                 case NC:
                     p = new Pair<String, Integer>(host, port);
                     if (!ncs.contains(host)) {
-                        throw new IllegalArgumentException("Invalid NC " + host
-                                + " as it is not part of the AsterixDB cluster. Valid choices are "
-                                + StringUtils.join(ncs, ", "));
+                        throw new IllegalArgumentException(
+                                "Invalid NC " + host + " as it is not part of the AsterixDB cluster. Valid choices are "
+                                        + StringUtils.join(ncs, ", "));
 
                     }
                     break;
@@ -152,21 +151,4 @@
     public ARecordType getAdapterOutputType() {
         return outputType;
     }
-
-    @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.UNKNOWN;
-    }
-
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        throw new UnsupportedOperationException("Tracking of ingested records not enabled");
-    }
-    
-    public void setFiles(List<ExternalFile> files) throws AlgebricksException {
-        throw new AlgebricksException("files access not supported for this adapter");
-    }
 }
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapter.java
similarity index 88%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapter.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapter.java
index e537ef7..db38c12 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapter.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -25,10 +25,10 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.hyracks.api.comm.IFrameWriter;
 
-public class SocketClientAdapter implements IFeedAdapter {
+public class SocketClientAdapter implements IDataSourceAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -93,17 +93,13 @@
     }
 
     @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PUSH;
-    }
-
-    @Override
-    public void stop() throws Exception {
+    public boolean stop() throws Exception {
         continueStreaming = false;
+        return true;
     }
 
     @Override
-    public boolean handleException(Exception e) {
+    public boolean handleException(Throwable e) {
         return false;
     }
 
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapterFactory.java
similarity index 76%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapterFactory.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapterFactory.java
index 3d2f5af..a1e90a8 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/SocketClientAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/SocketClientAdapterFactory.java
@@ -16,19 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.util.Map;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
-public class SocketClientAdapterFactory implements IFeedAdapterFactory {
+public class SocketClientAdapterFactory implements IAdapterFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -54,13 +54,8 @@
     }
 
     @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public String getName() {
-        return "socket_client";
+    public String getAlias() {
+        return ExternalDataConstants.ALIAS_SOCKET_CLIENT_ADAPTER;
     }
 
     @Override
@@ -69,7 +64,7 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+    public IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
         Pair<String, Integer> socket = genericSocketAdapterFactory.getSockets().get(partition);
         return new SocketClientAdapter(socket.second, fileSplits[partition]);
     }
@@ -79,14 +74,4 @@
         return outputType;
     }
 
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
 }
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TweetGenerator.java b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/TweetGenerator.java
similarity index 94%
rename from asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TweetGenerator.java
rename to asterix-external-data/src/main/java/org/apache/asterix/external/runtime/TweetGenerator.java
index 85195fb..b5fd454 100644
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/TweetGenerator.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/runtime/TweetGenerator.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.tools.external.data;
+package org.apache.asterix.external.runtime;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -27,9 +27,9 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.tools.external.data.DataGenerator.InitializationInfo;
-import org.apache.asterix.tools.external.data.DataGenerator.TweetMessage;
-import org.apache.asterix.tools.external.data.DataGenerator.TweetMessageIterator;
+import org.apache.asterix.external.runtime.DataGenerator.InitializationInfo;
+import org.apache.asterix.external.runtime.DataGenerator.TweetMessage;
+import org.apache.asterix.external.runtime.DataGenerator.TweetMessageIterator;
 
 public class TweetGenerator {
 
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolver.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolver.java
deleted file mode 100644
index a897294..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolver.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.util;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.util.AsterixRuntimeUtil;
-
-/**
- * Resolves a value (DNS/IP Address) to the id of a Node Controller running at the location.
- */
-public class DNSResolver implements INodeResolver {
-
-    private static Random random = new Random();
-
-    @Override
-    public String resolveNode(String value) throws AsterixException {
-        try {
-            InetAddress ipAddress = InetAddress.getByName(value);
-            Set<String> nodeControllers = AsterixRuntimeUtil.getNodeControllersOnIP(ipAddress);
-            if (nodeControllers == null || nodeControllers.isEmpty()) {
-                throw new AsterixException(" No node controllers found at the address: " + value);
-            }
-            String chosenNCId = nodeControllers.toArray(new String[] {})[random.nextInt(nodeControllers.size())];
-            return chosenNCId;
-        }catch (UnknownHostException e){
-            throw new AsterixException("Unable to resolve hostname '"+ value + "' to an IP address");
-        } catch (AsterixException ae) {
-            throw ae;
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-    }
-}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolverFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolverFactory.java
index 6862d7a..f8585bb 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolverFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DNSResolverFactory.java
@@ -18,12 +18,15 @@
  */
 package org.apache.asterix.external.util;
 
+import org.apache.asterix.external.api.INodeResolver;
+import org.apache.asterix.external.api.INodeResolverFactory;
+
 /**
- * Factory for creating instance of {@link DNSResolver}
+ * Factory for creating instance of {@link NodeResolver}
  */
 public class DNSResolverFactory implements INodeResolverFactory {
 
-    private static final INodeResolver INSTANCE = new DNSResolver();
+    private static final INodeResolver INSTANCE = new NodeResolver();
 
     @Override
     public INodeResolver createNodeResolver() {
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
new file mode 100644
index 0000000..ea13f25
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.parse.ITupleForwarder;
+import org.apache.asterix.common.parse.ITupleForwarder.TupleForwardPolicy;
+import org.apache.asterix.external.dataflow.CounterTimerTupleForwarder;
+import org.apache.asterix.external.dataflow.FrameFullTupleForwarder;
+import org.apache.asterix.external.dataflow.RateControlledTupleForwarder;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class DataflowUtils {
+    public static void addTupleToFrame(FrameTupleAppender appender, ArrayTupleBuilder tb, IFrameWriter writer)
+            throws HyracksDataException {
+        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+            appender.flush(writer, true);
+            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    public static ITupleForwarder getTupleForwarder(Map<String, String> configuration) throws AsterixException {
+        ITupleForwarder policy = null;
+        ITupleForwarder.TupleForwardPolicy policyType = null;
+        String propValue = configuration.get(ITupleForwarder.FORWARD_POLICY);
+        if (propValue == null) {
+            policyType = TupleForwardPolicy.FRAME_FULL;
+        } else {
+            policyType = TupleForwardPolicy.valueOf(propValue.trim().toUpperCase());
+        }
+        switch (policyType) {
+            case FRAME_FULL:
+                policy = new FrameFullTupleForwarder();
+                break;
+            case COUNTER_TIMER_EXPIRED:
+                policy = new CounterTimerTupleForwarder();
+                break;
+            case RATE_CONTROLLED:
+                policy = new RateControlledTupleForwarder();
+                break;
+            default:
+                throw new AsterixException("Unknown tuple forward policy");
+        }
+        return policy;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java
new file mode 100644
index 0000000..7f91a2b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.api.IExternalDataSourceFactory;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.om.types.ARecordType;
+
+public class ExternalDataCompatibilityUtils {
+
+    public static void validateCompatibility(IExternalDataSourceFactory dataSourceFactory,
+            IDataParserFactory dataParserFactory) throws AsterixException {
+        if (dataSourceFactory.getDataSourceType() != dataParserFactory.getDataSourceType()) {
+            throw new AsterixException(
+                    "datasource-parser mismatch. datasource produces " + dataSourceFactory.getDataSourceType()
+                            + " and parser expects " + dataParserFactory.getDataSourceType());
+        }
+        if (dataSourceFactory.getDataSourceType() == DataSourceType.RECORDS) {
+            IRecordReaderFactory<?> recordReaderFactory = (IRecordReaderFactory<?>) dataSourceFactory;
+            IRecordDataParserFactory<?> recordParserFactory = (IRecordDataParserFactory<?>) dataParserFactory;
+            if (!recordParserFactory.getRecordClass().isAssignableFrom(recordReaderFactory.getRecordClass())) {
+                throw new AsterixException("datasource-parser mismatch. datasource produces records of type "
+                        + recordReaderFactory.getRecordClass() + " and parser expects records of type "
+                        + recordParserFactory.getRecordClass());
+            }
+        }
+    }
+
+    //TODO:Add remaining aliases
+    public static void addCompatabilityParameters(String adapterClassname, ARecordType itemType,
+            Map<String, String> configuration) throws AsterixException {
+        if (adapterClassname.equals(ExternalDataConstants.ALIAS_HDFS_ADAPTER)
+                || adapterClassname.equalsIgnoreCase(ExternalDataConstants.ADAPTER_HDFS_CLASSNAME)) {
+            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+                throw new AsterixException("Unspecified format parameter for HDFS adapter");
+            }
+            if (configuration.get(ExternalDataConstants.KEY_FORMAT).equals(ExternalDataConstants.FORMAT_BINARY)
+                    || configuration.get(ExternalDataConstants.KEY_FORMAT).equals(ExternalDataConstants.FORMAT_HIVE)) {
+                configuration.put(ExternalDataConstants.KEY_READER, ExternalDataConstants.READER_HDFS);
+            } else {
+                configuration.put(ExternalDataConstants.KEY_READER,
+                        configuration.get(ExternalDataConstants.KEY_FORMAT));
+                configuration.put(ExternalDataConstants.KEY_READER_STREAM, ExternalDataConstants.ALIAS_HDFS_ADAPTER);
+            }
+        }
+        if (adapterClassname.equals(ExternalDataConstants.ALIAS_LOCALFS_ADAPTER)
+                || adapterClassname.contains(ExternalDataConstants.ADAPTER_LOCALFS_CLASSNAME)) {
+            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+                throw new AsterixException("Unspecified format parameter for local file system adapter");
+            }
+            configuration.put(ExternalDataConstants.KEY_READER, configuration.get(ExternalDataConstants.KEY_FORMAT));
+            configuration.put(ExternalDataConstants.KEY_READER_STREAM, ExternalDataConstants.ALIAS_LOCALFS_ADAPTER);
+        }
+        if (configuration.get(ExternalDataConstants.KEY_PARSER) != null
+                && configuration.get(ExternalDataConstants.KEY_PARSER).equals(ExternalDataConstants.PARSER_HIVE)) {
+            configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_HIVE);
+        }
+        if (configuration.get(ExternalDataConstants.KEY_FILESYSTEM) != null) {
+            configuration.put(ExternalDataConstants.KEY_STREAM,
+                    configuration.get(ExternalDataConstants.KEY_FILESYSTEM));
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
new file mode 100644
index 0000000..2050e6a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+public class ExternalDataConstants {
+    //TODO: Remove unused variables.
+    /**
+     * Keys
+     */
+    // used to specify the stream factory for an adapter that has a stream data source
+    public static final String KEY_STREAM = "stream";
+    // used to specify the dataverse of the adapter
+    public static final String KEY_DATAVERSE = "dataverse";
+    // used to specify the socket addresses when reading data from sockets
+    public static final String KEY_SOCKETS = "sockets";
+    // specify whether the socket address points to an NC or an IP
+    public static final String KEY_MODE = "address-type";
+    // specify the hdfs name node address when reading hdfs data
+    public static final String KEY_HDFS_URL = "hdfs";
+    // specify the path when reading from a file system
+    public static final String KEY_PATH = "path";
+    public static final String KEY_INPUT_FORMAT = "input-format";
+    public static final String KEY_FILESYSTEM = "fs";
+    public static final String KEY_HADOOP_FILESYSTEM_URI = "fs.defaultFS";
+    public static final String KEY_HADOOP_FILESYSTEM_CLASS = "fs.hdfs.impl";
+    public static final String KEY_HADOOP_INPUT_DIR = "mapred.input.dir";
+    public static final String KEY_HADOOP_INPUT_FORMAT = "mapred.input.format.class";
+    public static final String KEY_HADOOP_SHORT_CIRCUIT = "dfs.client.read.shortcircuit";
+    public static final String KEY_HADOOP_SOCKET_PATH = "dfs.domain.socket.path";
+    public static final String KEY_HADOOP_BUFFER_SIZE = "io.file.buffer.size";
+    public static final String KEY_SOURCE_DATATYPE = "type-name";
+    public static final String KEY_DELIMITER = "delimiter";
+    public static final String KEY_PARSER_FACTORY = "tuple-parser";
+    public static final String KEY_DATA_PARSER = "parser";
+    public static final String KEY_HEADER = "header";
+    public static final String KEY_READER = "reader";
+    public static final String KEY_READER_STREAM = "reader-stream";
+    public static final String KEY_TYPE_NAME = "type-name";
+    public static final String KEY_RECORD_START = "record-start";
+    public static final String KEY_RECORD_END = "record-end";
+    public static final String KEY_EXPRESSION = "expression";
+    public static final String KEY_LOCAL_SOCKET_PATH = "local-socket-path";
+    public static final String KEY_FORMAT = "format";
+    public static final String KEY_QUOTE = "quote";
+    public static final String KEY_PARSER = "parser";
+    public static final String KEY_DATASET_RECORD = "dataset-record";
+    public static final String KEY_HIVE_SERDE = "hive-serde";
+    public static final String KEY_RSS_URL = "url";
+    public static final String KEY_INTERVAL = "interval";
+    public static final String KEY_PULL = "pull";
+    public static final String KEY_PUSH = "push";
+    /**
+     * HDFS class names
+     */
+    public static final String CLASS_NAME_TEXT_INPUT_FORMAT = TextInputFormat.class.getName();
+    public static final String CLASS_NAME_SEQUENCE_INPUT_FORMAT = SequenceFileInputFormat.class.getName();
+    public static final String CLASS_NAME_RC_INPUT_FORMAT = RCFileInputFormat.class.getName();
+    public static final String CLASS_NAME_HDFS_FILESYSTEM = DistributedFileSystem.class.getName();
+    /**
+     * input formats aliases
+     */
+    public static final String INPUT_FORMAT_TEXT = "text-input-format";
+    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+    public static final String INPUT_FORMAT_RC = "rc-input-format";
+    /**
+     * Builtin streams
+     */
+
+    /**
+     * Builtin record readers
+     */
+    public static final String READER_HDFS = "hdfs";
+    public static final String READER_ADM = "adm";
+    public static final String READER_SEMISTRUCTURED = "semi-structured";
+    public static final String READER_DELIMITED = "delimited-text";
+
+    public static final String CLUSTER_LOCATIONS = "cluster-locations";
+    public static final String SCHEDULER = "hdfs-scheduler";
+    public static final String PARSER_HIVE = "hive-parser";
+    public static final String HAS_HEADER = "has.header";
+    public static final String TIME_TRACKING = "time.tracking";
+    public static final String DEFAULT_QUOTE = "\"";
+    public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+    public static final String DEFAULT_DELIMITER = ",";
+    public static final String EXTERNAL_LIBRARY_SEPARATOR = "#";
+    public static final String HDFS_INDEXING_ADAPTER = "hdfs-indexing-adapter";
+    /**
+     * supported builtin record formats
+     */
+    public static final String FORMAT_HIVE = "hive";
+    public static final String FORMAT_BINARY = "binary";
+    public static final String FORMAT_ADM = "adm";
+    public static final String FORMAT_JSON = "json";
+    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String FORMAT_TWEET = "tweet";
+    public static final String FORMAT_RSS = "rss";
+
+    /**
+     * input streams
+     */
+    public static final String STREAM_HDFS = "hdfs";
+    public static final String STREAM_LOCAL_FILESYSTEM = "localfs";
+    public static final String STREAM_SOCKET = "socket";
+
+    /**
+     * adapter aliases
+     */
+    public static final String ALIAS_GENERIC_ADAPTER = "adapter";
+    public static final String ALIAS_LOCALFS_ADAPTER = "localfs";
+    public static final String ALIAS_HDFS_ADAPTER = "hdfs";
+    public static final String ALIAS_SOCKET_ADAPTER = "socket_adapter";
+    public static final String ALIAS_TWITTER_FIREHOSE_ADAPTER = "twitter_firehose";
+    public static final String ALIAS_SOCKET_CLIENT_ADAPTER = "socket_client";
+    public static final String ALIAS_RSS_ADAPTER = "rss_feed";
+    public static final String ALIAS_FILE_FEED_ADAPTER = "file_feed";
+    public static final String ALIAS_TWITTER_PUSH_ADAPTER = "push_twitter";
+    public static final String ALIAS_TWITTER_PULL_ADAPTER = "pull_twitter";
+    public static final String ALIAS_TWITTER_AZURE_ADAPTER = "azure_twitter";
+    public static final String ALIAS_CNN_ADAPTER = "cnn_feed";
+
+    /**
+     * For backward compatability
+     */
+    public static final String ADAPTER_LOCALFS_CLASSNAME = "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter";
+    public static final String ADAPTER_HDFS_CLASSNAME = "org.apache.asterix.external.dataset.adapter.HDFSAdapter";
+
+    /**
+     * Constant characters
+     */
+    public static final char ESCAPE = '\\';
+    public static final char QUOTE = '"';
+    public static final char SPACE = ' ';
+    public static final char TAB = '\t';
+    public static final char LF = '\n';
+    public static final char CR = '\r';
+    public static final char DEFAULT_RECORD_START = '{';
+    public static final char DEFAULT_RECORD_END = '}';
+
+    /**
+     * Constant byte characters
+     */
+    public static final byte EOL = '\n';
+    public static final byte BYTE_CR = '\r';
+    /**
+     * Size default values
+     */
+    public static final int DEFAULT_BUFFER_SIZE = 4096;
+    public static final int DEFAULT_BUFFER_INCREMENT = 4096;
+
+    /**
+     * Expected parameter values
+     */
+    public static final String PARAMETER_OF_SIZE_ONE = "Value of size 1";
+
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java
new file mode 100644
index 0000000..9dcaef4
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+public class ExternalDataExceptionUtils {
+    public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
+    public static final String MISSING_PARAMETER = "Missing parameter.\n";
+    public static final String PARAMETER_NAME = "Parameter name: ";
+    public static final String EXPECTED_VALUE = "Expected value: ";
+    public static final String PASSED_VALUE = "Passed value: ";
+
+    public static String incorrectParameterMessage(String parameterName, String expectedValue, String passedValue) {
+        return INCORRECT_PARAMETER + PARAMETER_NAME + parameterName + ExternalDataConstants.LF + EXPECTED_VALUE
+                + expectedValue + ExternalDataConstants.LF + PASSED_VALUE + passedValue;
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
new file mode 100644
index 0000000..7c1c1b5
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.feeds.FeedConstants;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.library.ExternalLibraryManager;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+
+public class ExternalDataUtils {
+
+    // Get a delimiter from the given configuration
+    public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
+        String delimiterValue = configuration.get(ExternalDataConstants.KEY_DELIMITER);
+        if (delimiterValue == null) {
+            delimiterValue = ExternalDataConstants.DEFAULT_DELIMITER;
+        } else if (delimiterValue.length() != 1) {
+            throw new AsterixException(
+                    "'" + delimiterValue + "' is not a valid delimiter. The length of a delimiter should be 1.");
+        }
+        return delimiterValue.charAt(0);
+    }
+
+    // Get a quote from the given configuration when the delimiter is given
+    // Need to pass delimiter to check whether they share the same character
+    public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
+        String quoteValue = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        if (quoteValue == null) {
+            quoteValue = ExternalDataConstants.DEFAULT_QUOTE;
+        } else if (quoteValue.length() != 1) {
+            throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
+        }
+
+        // Since delimiter (char type value) can't be null,
+        // we only check whether delimiter and quote use the same character
+        if (quoteValue.charAt(0) == delimiter) {
+            throw new AsterixException(
+                    "Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter + "'. ");
+        }
+
+        return quoteValue.charAt(0);
+    }
+
+    // Get the header flag
+    public static boolean getHasHeader(Map<String, String> configuration) {
+        return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_HEADER));
+    }
+
+    public static DataSourceType getDataSourceType(Map<String, String> configuration) throws AsterixException {
+        if (isDataSourceStreamProvider(configuration)) {
+            return DataSourceType.STREAM;
+        } else if (isDataSourceRecordReader(configuration)) {
+            return DataSourceType.RECORDS;
+        } else {
+            throw new AsterixException(
+                    "unable to determine whether input is a stream provider or a record reader. parameters: "
+                            + ExternalDataConstants.KEY_STREAM + " or " + ExternalDataConstants.KEY_READER
+                            + " must be specified");
+        }
+    }
+
+    public static boolean isExternal(String aString) {
+        return (aString.contains(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR) && aString.trim().length() > 1);
+    }
+
+    public static ClassLoader getClassLoader(String dataverse, String library) {
+        return ExternalLibraryManager.getLibraryClassLoader(dataverse, library);
+    }
+
+    public static String getLibraryName(String aString) {
+        return aString.trim().split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[0];
+    }
+
+    public static String getExternalClassName(String aString) {
+        return aString.trim().split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[1];
+    }
+
+    public static IInputStreamProviderFactory createExternalInputStreamFactory(String dataverse, String stream)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        String libraryName = getLibraryName(stream);
+        String className = getExternalClassName(stream);
+        ClassLoader classLoader = getClassLoader(dataverse, libraryName);
+        return ((IInputStreamProviderFactory) (classLoader.loadClass(className).newInstance()));
+    }
+
+    public static String getDataverse(Map<String, String> configuration) {
+        return configuration.get(ExternalDataConstants.KEY_DATAVERSE);
+    }
+
+    public static boolean isDataSourceStreamProvider(Map<String, String> configuration) {
+        return configuration.containsKey(ExternalDataConstants.KEY_STREAM);
+    }
+
+    private static boolean isDataSourceRecordReader(Map<String, String> configuration) {
+        return configuration.containsKey(ExternalDataConstants.KEY_READER);
+    }
+
+    public static String getRecordFormat(Map<String, String> configuration) {
+        String parserFormat = configuration.get(ExternalDataConstants.KEY_DATA_PARSER);
+        return parserFormat != null ? parserFormat : configuration.get(ExternalDataConstants.KEY_FORMAT);
+    }
+
+    private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
+
+    private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
+        Map<ATypeTag, IValueParserFactory> m = new HashMap<ATypeTag, IValueParserFactory>();
+        m.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
+        m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+        m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+        m.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
+        m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+        return m;
+    }
+
+    public static IValueParserFactory[] getValueParserFactories(ARecordType recordType) {
+        int n = recordType.getFieldTypes().length;
+        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+        for (int i = 0; i < n; i++) {
+            ATypeTag tag = null;
+            if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
+                List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
+                if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
+                    throw new NotImplementedException("Non-optional UNION type is not supported.");
+                }
+                tag = unionTypes.get(1).getTypeTag();
+            } else {
+                tag = recordType.getFieldTypes()[i].getTypeTag();
+            }
+            if (tag == null) {
+                throw new NotImplementedException("Failed to get the type information for field " + i + ".");
+            }
+            IValueParserFactory vpf = valueParserFactoryMap.get(tag);
+            if (vpf == null) {
+                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+            }
+            fieldParserFactories[i] = vpf;
+        }
+        return fieldParserFactories;
+    }
+
+    public static String getRecordReaderStreamName(Map<String, String> configuration) {
+        return configuration.get(ExternalDataConstants.KEY_READER_STREAM);
+    }
+
+    public static boolean hasHeader(Map<String, String> configuration) {
+        String value = configuration.get(ExternalDataConstants.KEY_HEADER);
+        if (value != null) {
+            return Boolean.valueOf(value);
+        }
+        return false;
+    }
+
+    public static boolean isPull(Map<String, String> configuration) {
+        String pull = configuration.get(ExternalDataConstants.KEY_PULL);
+        if (pull == null) {
+            return false;
+        }
+        return Boolean.parseBoolean(pull);
+    }
+
+    public static boolean isPush(Map<String, String> configuration) {
+        String push = configuration.get(ExternalDataConstants.KEY_PUSH);
+        if (push == null) {
+            return false;
+        }
+        return Boolean.parseBoolean(push);
+    }
+
+    public static IRecordReaderFactory<?> createExternalRecordReaderFactory(String dataverse, String reader)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        String library = reader.substring(0, reader.indexOf(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR));
+        ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(dataverse, library);
+        return (IRecordReaderFactory<?>) classLoader
+                .loadClass(reader.substring(reader.indexOf(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR) + 1))
+                .newInstance();
+    }
+
+    public static IDataParserFactory createExternalParserFactory(String dataverse, String parserFactoryName)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        String library = parserFactoryName.substring(0,
+                parserFactoryName.indexOf(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR));
+        ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(dataverse, library);
+        return (IDataParserFactory) classLoader
+                .loadClass(parserFactoryName
+                        .substring(parserFactoryName.indexOf(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR) + 1))
+                .newInstance();
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
new file mode 100644
index 0000000..de6737a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.IndexingScheduler;
+import org.apache.asterix.external.indexing.RecordId.RecordIdType;
+import org.apache.asterix.external.input.stream.HDFSInputStreamProvider;
+import org.apache.asterix.om.util.AsterixAppContextInfo;
+import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.ICCContext;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.hdfs.scheduler.Scheduler;
+
+public class HDFSUtils {
+
+    public static Scheduler initializeHDFSScheduler() {
+        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+        Scheduler scheduler = null;
+        try {
+            scheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
+                    ccContext.getClusterControllerInfo().getClientNetPort());
+        } catch (HyracksException e) {
+            throw new IllegalStateException("Cannot obtain hdfs scheduler");
+        }
+        return scheduler;
+    }
+
+    public static IndexingScheduler initializeIndexingHDFSScheduler() {
+        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+        IndexingScheduler scheduler = null;
+        try {
+            scheduler = new IndexingScheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
+                    ccContext.getClusterControllerInfo().getClientNetPort());
+        } catch (HyracksException e) {
+            throw new IllegalStateException("Cannot obtain hdfs scheduler");
+        }
+        return scheduler;
+    }
+
+    /**
+     * Instead of creating the split using the input format, we do it manually
+     * This function returns fileSplits (1 per hdfs file block) irrespective of the number of partitions
+     * and the produced splits only cover intersection between current files in hdfs and files stored internally
+     * in AsterixDB
+     * 1. NoOp means appended file
+     * 2. AddOp means new file
+     * 3. UpdateOp means the delta of a file
+     * @return
+     * @throws IOException
+     */
+    public static InputSplit[] getSplits(JobConf conf, List<ExternalFile> files) throws IOException {
+        // Create file system object
+        FileSystem fs = FileSystem.get(conf);
+        ArrayList<FileSplit> fileSplits = new ArrayList<FileSplit>();
+        ArrayList<ExternalFile> orderedExternalFiles = new ArrayList<ExternalFile>();
+        // Create files splits
+        for (ExternalFile file : files) {
+            Path filePath = new Path(file.getFileName());
+            FileStatus fileStatus;
+            try {
+                fileStatus = fs.getFileStatus(filePath);
+            } catch (FileNotFoundException e) {
+                // file was deleted at some point, skip to next file
+                continue;
+            }
+            if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP
+                    && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
+                // Get its information from HDFS name node
+                BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, file.getSize());
+                // Create a split per block
+                for (BlockLocation block : fileBlocks) {
+                    if (block.getOffset() < file.getSize()) {
+                        fileSplits
+                                .add(new FileSplit(filePath,
+                                        block.getOffset(), (block.getLength() + block.getOffset()) < file.getSize()
+                                                ? block.getLength() : (file.getSize() - block.getOffset()),
+                                block.getHosts()));
+                        orderedExternalFiles.add(file);
+                    }
+                }
+            } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP
+                    && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
+                long oldSize = 0L;
+                long newSize = file.getSize();
+                for (int i = 0; i < files.size(); i++) {
+                    if (files.get(i).getFileName() == file.getFileName() && files.get(i).getSize() != file.getSize()) {
+                        newSize = files.get(i).getSize();
+                        oldSize = file.getSize();
+                        break;
+                    }
+                }
+
+                // Get its information from HDFS name node
+                BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, newSize);
+                // Create a split per block
+                for (BlockLocation block : fileBlocks) {
+                    if (block.getOffset() + block.getLength() > oldSize) {
+                        if (block.getOffset() < newSize) {
+                            // Block interact with delta -> Create a split
+                            long startCut = (block.getOffset() > oldSize) ? 0L : oldSize - block.getOffset();
+                            long endCut = (block.getOffset() + block.getLength() < newSize) ? 0L
+                                    : block.getOffset() + block.getLength() - newSize;
+                            long splitLength = block.getLength() - startCut - endCut;
+                            fileSplits.add(new FileSplit(filePath, block.getOffset() + startCut, splitLength,
+                                    block.getHosts()));
+                            orderedExternalFiles.add(file);
+                        }
+                    }
+                }
+            }
+        }
+        fs.close();
+        files.clear();
+        files.addAll(orderedExternalFiles);
+        return fileSplits.toArray(new FileSplit[fileSplits.size()]);
+    }
+
+    public static String getInputFormatClassName(Map<String, String> configuration) {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        switch (inputFormatParameter) {
+            case ExternalDataConstants.INPUT_FORMAT_TEXT:
+                return ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT;
+            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
+                return ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT;
+            case ExternalDataConstants.INPUT_FORMAT_RC:
+                return ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT;
+            default:
+                return inputFormatParameter;
+        }
+    }
+
+    public static Class<?> getInputFormatClass(Map<String, String> configuration) throws ClassNotFoundException {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        switch (inputFormatParameter) {
+            case ExternalDataConstants.INPUT_FORMAT_TEXT:
+                return TextInputFormat.class;
+            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
+                return SequenceFileInputFormat.class;
+            case ExternalDataConstants.INPUT_FORMAT_RC:
+                return RCFileInputFormat.class;
+            default:
+                return Class.forName(inputFormatParameter);
+        }
+    }
+
+    public static JobConf configureHDFSJobConf(Map<String, String> configuration) throws Exception {
+        JobConf conf = new JobConf();
+
+        String localShortCircuitSocketPath = configuration.get(ExternalDataConstants.KEY_LOCAL_SOCKET_PATH);
+        String formatClassName = HDFSUtils.getInputFormatClassName(configuration);
+        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI,
+                configuration.get(ExternalDataConstants.KEY_HDFS_URL).trim());
+        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_CLASS, ExternalDataConstants.CLASS_NAME_HDFS_FILESYSTEM);
+        conf.setClassLoader(HDFSInputStreamProvider.class.getClassLoader());
+        conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, configuration.get(ExternalDataConstants.KEY_PATH).trim());
+        conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_FORMAT, formatClassName);
+
+        // Enable local short circuit reads if user supplied the parameters
+        if (localShortCircuitSocketPath != null) {
+            conf.set(ExternalDataConstants.KEY_HADOOP_SHORT_CIRCUIT, "true");
+            conf.set(ExternalDataConstants.KEY_HADOOP_SOCKET_PATH, localShortCircuitSocketPath.trim());
+        }
+        return conf;
+    }
+
+    public static AlgebricksPartitionConstraint getPartitionConstraints(
+            AlgebricksPartitionConstraint clusterLocations) {
+        if (clusterLocations == null) {
+            ArrayList<String> locs = new ArrayList<String>();
+            Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+            for (String i : stores.keySet()) {
+                int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+                for (int k = 0; k < numIODevices; k++) {
+                    locs.add(i);
+                }
+            }
+            String[] cluster = new String[locs.size()];
+            cluster = locs.toArray(cluster);
+            clusterLocations = new AlgebricksAbsolutePartitionConstraint(cluster);
+        }
+        return clusterLocations;
+    }
+
+    public static RecordIdType getRecordIdType(Map<String, String> configuration) {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        switch (inputFormatParameter) {
+            case ExternalDataConstants.INPUT_FORMAT_TEXT:
+            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
+                return RecordIdType.OFFSET;
+            case ExternalDataConstants.INPUT_FORMAT_RC:
+                return RecordIdType.RC;
+            default:
+                return null;
+        }
+    }
+}
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentitiyResolverFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentitiyResolverFactory.java
index 776061f..582189a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentitiyResolverFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentitiyResolverFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.util;
 
+import org.apache.asterix.external.api.INodeResolver;
+import org.apache.asterix.external.api.INodeResolverFactory;
+
 /**
  * Factory for creating an instance of @see {IdentityResolver}.
  * Identity resolver simply resolves a value to itself and is useful when value being resolved
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentityResolver.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentityResolver.java
index 2b792b2..bda5f1e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentityResolver.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/IdentityResolver.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.external.util;
 
+import org.apache.asterix.external.api.INodeResolver;
+
 /**
  * Identity resolver simply resolves a value to itself and is useful when value being resolved
  * is a node controller id.
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java
new file mode 100644
index 0000000..61764d7
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.INodeResolver;
+import org.apache.asterix.om.util.AsterixRuntimeUtil;
+
+/**
+ * Resolves a value (DNS/IP Address) or a (Node Controller Id) to the id of a Node Controller running at the location.
+ */
+public class NodeResolver implements INodeResolver {
+    //TODO: change this call and replace by calling AsterixClusterProperties
+    private static final Random random = new Random();
+    private static final Map<InetAddress, Set<String>> ncMap = new HashMap<InetAddress, Set<String>>();
+    private static final Set<String> ncs = new HashSet<String>();
+
+    @Override
+    public String resolveNode(String value) throws AsterixException {
+        UnknownHostException uhe = null;
+        try {
+            if (ncMap.isEmpty()) {
+                NodeResolver.updateNCs();
+            }
+            InetAddress ipAddress = null;
+            try {
+                ipAddress = InetAddress.getByName(value);
+            } catch (UnknownHostException e) {
+                uhe = e;
+            }
+            if (ipAddress == null) {
+                if (ncs.contains(value)) {
+                    return value;
+                } else {
+                    NodeResolver.updateNCs();
+                    if (ncs.contains(value)) {
+                        return value;
+                    } else {
+                        throw new AsterixException("address passed: '" + value
+                                + "' couldn't be resolved to an ip address and is not an NC id. Existing NCs are "
+                                + ncs.toString(), uhe);
+                    }
+                }
+
+            }
+            Set<String> nodeControllers = ncMap.get(ipAddress);
+            if (nodeControllers == null || nodeControllers.isEmpty()) {
+                throw new AsterixException(" No node controllers found at the address: " + value);
+            }
+            String chosenNCId = nodeControllers.toArray(new String[] {})[random.nextInt(nodeControllers.size())];
+            return chosenNCId;
+        } catch (UnknownHostException e) {
+            throw new AsterixException("Unable to resolve hostname '" + value + "' to an IP address");
+        } catch (AsterixException ae) {
+            throw ae;
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+    }
+
+    private static void updateNCs() throws Exception {
+        synchronized (ncMap) {
+            ncMap.clear();
+            AsterixRuntimeUtil.getNodeControllerMap(ncMap);
+            synchronized (ncs) {
+                ncs.clear();
+                for (Entry<InetAddress, Set<String>> entry : ncMap.entrySet()) {
+                    ncs.addAll(entry.getValue());
+                }
+            }
+        }
+    }
+}
diff --git a/asterix-runtime/src/main/resources/adm.grammar b/asterix-external-data/src/main/resources/adm.grammar
similarity index 100%
rename from asterix-runtime/src/main/resources/adm.grammar
rename to asterix-external-data/src/main/resources/adm.grammar
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFactory.java
index 2c1f02a..db693a1 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFactory.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalScalarFunction;
-import org.apache.asterix.external.library.IFunctionFactory;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class AddHashTagsFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
index bca508f..db717e6 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
@@ -23,6 +23,8 @@
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.external.library.java.JObjects.JString;
 import org.apache.asterix.external.library.java.JObjects.JUnorderedList;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 import org.apache.asterix.external.util.Datatypes;
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFactory.java
index aec9e5d..a13da84 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class AddHashTagsInPlaceFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFunction.java
index 2765225..399f0f9 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsInPlaceFunction.java
@@ -21,6 +21,8 @@
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.external.library.java.JObjects.JString;
 import org.apache.asterix.external.library.java.JObjects.JUnorderedList;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 import org.apache.asterix.external.util.Datatypes;
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFactory.java
index dc0ab7a..9050462 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFactory.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalFunction;
-import org.apache.asterix.external.library.IFunctionFactory;
+import org.apache.asterix.external.api.IExternalFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class AllTypesFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFunction.java
index 12ce871..8f65bee 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/AllTypesFunction.java
@@ -35,6 +35,8 @@
 import org.apache.asterix.external.library.java.JObjects.JString;
 import org.apache.asterix.external.library.java.JObjects.JTime;
 import org.apache.asterix.external.library.java.JObjects.JUnorderedList;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 
 public class AllTypesFunction implements IExternalScalarFunction {
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFactory.java
index 7d1d3da..e15cb3d 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFactory.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalScalarFunction;
-import org.apache.asterix.external.library.IFunctionFactory;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class CapitalFinderFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFunction.java
index 21467af..969e109 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/CapitalFinderFunction.java
@@ -23,6 +23,8 @@
 
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.external.library.java.JObjects.JString;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 
 public class CapitalFinderFunction implements IExternalScalarFunction {
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFactory.java
index 21ad776..5d8126b 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class EchoDelayFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFunction.java
index e564ca0..c115ac4 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/EchoDelayFunction.java
@@ -20,6 +20,8 @@
 
 import java.util.Random;
 
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 
 public class EchoDelayFunction implements IExternalScalarFunction {
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFactory.java
index db3a5fa..5515ebd 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFactory.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalScalarFunction;
-import org.apache.asterix.external.library.IFunctionFactory;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class ParseTweetFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFunction.java
index caa0544..b9c736a 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/ParseTweetFunction.java
@@ -21,6 +21,8 @@
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.external.library.java.JObjects.JString;
 import org.apache.asterix.external.library.java.JObjects.JUnorderedList;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 
 public class ParseTweetFunction implements IExternalScalarFunction {
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFactory.java
index 2ccc91c..5202093 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class SumFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFunction.java
index d2c9e1b..d81f01b 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/SumFunction.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalScalarFunction;
-import org.apache.asterix.external.library.IFunctionHelper;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JObjects.JInt;
 
 public class SumFunction implements IExternalScalarFunction {
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFactory.java
index 0d738da..f74ed38 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFactory.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.library;
 
-import org.apache.asterix.external.library.IExternalFunction;
-import org.apache.asterix.external.library.IFunctionFactory;
+import org.apache.asterix.external.api.IExternalFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
 
 public class UpperCaseFactory implements IFunctionFactory {
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFunction.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFunction.java
index 56121b0..70bd3e1 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFunction.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/UpperCaseFunction.java
@@ -23,6 +23,8 @@
 import org.apache.asterix.external.library.java.JObjects.JInt;
 import org.apache.asterix.external.library.java.JObjects.JRecord;
 import org.apache.asterix.external.library.java.JObjects.JString;
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.library.java.JTypeTag;
 
 /**
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
index 39f8271..df0fb94 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
@@ -27,14 +27,13 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
 import org.apache.asterix.external.dataset.adapter.StreamBasedAdapter;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public class TestTypedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+public class TestTypedAdapter extends StreamBasedAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -126,17 +125,13 @@
     }
 
     @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PUSH;
-    }
-
-    @Override
-    public void stop() throws Exception {
+    public boolean stop() throws Exception {
         generator.stop();
+        return true;
     }
 
     @Override
-    public boolean handleException(Exception e) {
+    public boolean handleException(Throwable e) {
         return false;
     }
 
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
index c177a58..6b08f3a 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
@@ -18,26 +18,30 @@
  */
 package org.apache.asterix.external.library.adapter;
 
+import java.io.InputStream;
 import java.util.Map;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory.SupportedOperation;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.common.parse.ITupleForwarder;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.parser.ADMDataParser;
+import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.std.file.ITupleParser;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public class TestTypedAdapterFactory implements IFeedAdapterFactory {
+public class TestTypedAdapterFactory implements IAdapterFactory {
 
     private static final long serialVersionUID = 1L;
 
-    public static final String NAME = "test_typed_adapter";
-
     private ARecordType outputType;
 
     public static final String KEY_NUM_OUTPUT_RECORDS = "num_output_records";
@@ -45,13 +49,8 @@
     private Map<String, String> configuration;
 
     @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public String getName() {
-        return NAME;
+    public String getAlias() {
+        return "test_typed";
     }
 
     @Override
@@ -60,9 +59,47 @@
     }
 
     @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        ITupleParserFactory tupleParserFactory = new AsterixTupleParserFactory(configuration, outputType,
-                InputDataFormat.ADM);
+    public IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+        ITupleParserFactory tupleParserFactory = new ITupleParserFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ITupleParser createTupleParser(final IHyracksCommonContext ctx) throws HyracksDataException {
+                ADMDataParser parser;
+                ITupleForwarder forwarder;
+                ArrayTupleBuilder tb;
+                try {
+                    parser = new ADMDataParser();
+                    forwarder = DataflowUtils.getTupleForwarder(configuration);
+                    forwarder.configure(configuration);
+                    tb = new ArrayTupleBuilder(1);
+                } catch (AsterixException e) {
+                    throw new HyracksDataException(e);
+                }
+                return new ITupleParser() {
+
+                    @Override
+                    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+                        try {
+                            parser.configure(configuration, outputType);
+                            parser.setInputStream(in);
+                            forwarder.initialize(ctx, writer);
+                            while (true) {
+                                tb.reset();
+                                if (!parser.parse(tb.getDataOutput())) {
+                                    break;
+                                }
+                                tb.addFieldEndOffset();
+                                forwarder.addTuple(tb);
+                            }
+                            forwarder.close();
+                        } catch (Exception e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+                };
+            }
+        };
         return new TestTypedAdapter(tupleParserFactory, outputType, ctx, configuration, partition);
     }
 
@@ -77,14 +114,4 @@
         this.outputType = outputType;
     }
 
-    @Override
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    @Override
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        return null;
-    }
-
 }
diff --git a/asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java b/asterix-external-data/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
similarity index 98%
rename from asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
rename to asterix-external-data/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
index e23e255..698e414 100644
--- a/asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
@@ -23,8 +23,8 @@
 import java.io.DataOutputStream;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.asterix.external.parser.ADMDataParser;
 import org.apache.asterix.om.base.AMutableInterval;
-import org.apache.asterix.runtime.operators.file.ADMDataParser;
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
index 0613498..42827b4 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
@@ -20,7 +20,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.test.runtime.HDFSCluster;
@@ -43,8 +43,8 @@
     protected static final Logger LOGGER = Logger.getLogger(AbstractExecutionIT.class.getName());
 
     protected static final String PATH_ACTUAL = "ittest" + File.separator;
-    protected static final String PATH_BASE = StringUtils.join(new String[] { "..", "asterix-app", "src", "test",
-            "resources", "runtimets" }, File.separator);
+    protected static final String PATH_BASE = StringUtils
+            .join(new String[] { "..", "asterix-app", "src", "test", "resources", "runtimets" }, File.separator);
 
     protected static final String HDFS_BASE = "../asterix-app/";
 
@@ -63,21 +63,21 @@
 
         //This is nasty but there is no very nice way to set a system property on each NC that I can figure.
         //The main issue is that we need the NC resolver to be the IdentityResolver and not the DNSResolver.
-        FileUtils.copyFile(
-                new File(StringUtils.join(new String[] { "src", "test", "resources", "integrationts", "asterix-configuration.xml" }, File.separator)),
+        FileUtils
+                .copyFile(
+                        new File(StringUtils.join(new String[] { "src", "test", "resources", "integrationts",
+                                "asterix-configuration.xml" }, File.separator)),
                 new File(AsterixInstallerIntegrationUtil.getManagixHome() + "/conf/asterix-configuration.xml"));
 
         AsterixLifecycleIT.setUp();
 
-
         FileUtils.copyDirectoryStructure(
                 new File(StringUtils.join(new String[] { "..", "asterix-app", "data" }, File.separator)),
                 new File(AsterixInstallerIntegrationUtil.getManagixHome() + "/clusters/local/working_dir/data"));
 
-
         // Set the node resolver to be the identity resolver that expects node names
         // to be node controller ids; a valid assumption in test environment.
-        System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
+        System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
     }
 
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
index 438bb05..1da01c3 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
@@ -43,16 +43,21 @@
 
     @BeforeClass
     public static void setUp() throws Exception {
-        AsterixInstallerIntegrationUtil.init();
-        File asterixInstallerProjectDir = new File(System.getProperty("user.dir"));
-        String asterixExternalLibraryPath = asterixInstallerProjectDir.getParentFile().getAbsolutePath()
-                + File.separator + LIBRARY_PATH;
-        LOGGER.info("Installing library :" + LIBRARY_NAME + " located at " + asterixExternalLibraryPath
-                + " in dataverse " + LIBRARY_DATAVERSE);
-        AsterixInstallerIntegrationUtil.installLibrary(LIBRARY_NAME, LIBRARY_DATAVERSE, asterixExternalLibraryPath);
-        AsterixInstallerIntegrationUtil.transformIntoRequiredState(State.ACTIVE);
-        TestCaseContext.Builder b = new TestCaseContext.Builder();
-        testCaseCollection = b.build(new File(PATH_BASE));
+        try {
+            AsterixInstallerIntegrationUtil.init();
+            File asterixInstallerProjectDir = new File(System.getProperty("user.dir"));
+            String asterixExternalLibraryPath = asterixInstallerProjectDir.getParentFile().getAbsolutePath()
+                    + File.separator + LIBRARY_PATH;
+            LOGGER.info("Installing library :" + LIBRARY_NAME + " located at " + asterixExternalLibraryPath
+                    + " in dataverse " + LIBRARY_DATAVERSE);
+            AsterixInstallerIntegrationUtil.installLibrary(LIBRARY_NAME, LIBRARY_DATAVERSE, asterixExternalLibraryPath);
+            AsterixInstallerIntegrationUtil.transformIntoRequiredState(State.ACTIVE);
+            TestCaseContext.Builder b = new TestCaseContext.Builder();
+            testCaseCollection = b.build(new File(PATH_BASE));
+        } catch (Throwable th) {
+            th.printStackTrace();
+            throw th;
+        }
     }
 
     @AfterClass
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index c000d55..34a8733 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -93,6 +93,7 @@
         String command = "shutdown";
         cmdHandler.processCommand(command.split(" "));
 
+        //TODO: This must be fixed, an arbitrary wait for 2s is not a reliable way to make sure the process have completed successfully.
         Thread.sleep(2000);
 
         // start zookeeper
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ClusterExecutionIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ClusterExecutionIT.java
index 93e9f6dc..cf69e1a 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ClusterExecutionIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ClusterExecutionIT.java
@@ -18,10 +18,12 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.logging.Level;
-import java.util.logging.Logger;
 
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.test.runtime.HDFSCluster;
+import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.commons.lang3.StringUtils;
 import org.codehaus.plexus.util.FileUtils;
 import org.junit.AfterClass;
@@ -31,20 +33,16 @@
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.testframework.context.TestCaseContext;
-
 /**
  * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
  */
 @RunWith(Parameterized.class)
-public class ClusterExecutionIT extends AbstractExecutionIT{
+public class ClusterExecutionIT extends AbstractExecutionIT {
 
     private static final String CLUSTER_CC_ADDRESS = "10.10.0.2";
     private static final int CLUSTER_CC_API_PORT = 19002;
 
-    private final static TestExecutor testExecutor = new TestExecutor(CLUSTER_CC_ADDRESS,CLUSTER_CC_API_PORT);
+    private final static TestExecutor testExecutor = new TestExecutor(CLUSTER_CC_ADDRESS, CLUSTER_CC_API_PORT);
 
     @BeforeClass
     public static void setUp() throws Exception {
@@ -60,13 +58,14 @@
         AsterixClusterLifeCycleIT.setUp();
 
         FileUtils.copyDirectoryStructure(
-                new File(StringUtils.join(new String[] { "..", "asterix-app", "data" }, File.separator)), new File(
-                StringUtils.join(new String[] { "src", "test", "resources", "clusterts", "managix-working", "data" },
+                new File(StringUtils.join(new String[] { "..", "asterix-app", "data" }, File.separator)),
+                new File(StringUtils.join(
+                        new String[] { "src", "test", "resources", "clusterts", "managix-working", "data" },
                         File.separator)));
 
         // Set the node resolver to be the identity resolver that expects node names
         // to be node controller ids; a valid assumption in test environment.
-        System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
+        System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
     }
 
@@ -100,6 +99,7 @@
         this.tcCtx = tcCtx;
     }
 
+    @Override
     @Test
     public void test() throws Exception {
         testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false);
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixExecutionIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixExecutionIT.java
index 17184c7..492f173 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixExecutionIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixExecutionIT.java
@@ -14,25 +14,9 @@
  */
 package org.apache.asterix.installer.test;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.asterix.test.runtime.HDFSCluster;
-import org.codehaus.plexus.util.FileUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 /**
  * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
@@ -40,7 +24,6 @@
 @RunWith(Parameterized.class)
 public class ManagixExecutionIT extends AbstractExecutionIT {
 
-
     private TestCaseContext tcCtx;
 
     public ManagixExecutionIT(TestCaseContext tcCtx) {
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixSqlppExecutionIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixSqlppExecutionIT.java
index 2e66afd..b9c2072 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixSqlppExecutionIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ManagixSqlppExecutionIT.java
@@ -17,19 +17,8 @@
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.test.aql.TestExecutor;
-import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.lang3.StringUtils;
-import org.codehaus.plexus.util.FileUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -38,7 +27,7 @@
  * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
  */
 @RunWith(Parameterized.class)
-public class ManagixSqlppExecutionIT extends ManagixExecutionIT{
+public class ManagixSqlppExecutionIT extends ManagixExecutionIT {
 
     @Parameters
     public static Collection<Object[]> tests() throws Exception {
diff --git a/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java b/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
index a62abaa..71c762a 100644
--- a/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
+++ b/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.common.feeds.FeedId;
 import org.apache.asterix.common.feeds.FeedPolicyAccessor;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
@@ -188,7 +188,7 @@
         try {
             switch (feed.getFeedType()) {
                 case PRIMARY:
-                    Triple<IFeedAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
+                    Triple<IAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
 
                     factoryOutput = FeedUtil.getPrimaryFeedFactoryAndOutput((PrimaryFeed) feed, policyAccessor,
                             mdTxnCtx);
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 3e21653..a113864 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -72,11 +72,34 @@
 // ================================================================================
 //  Public interface
 // ================================================================================
-    
+
     public [LEXER_NAME](java.io.Reader stream) throws IOException{
         reInit(stream);
     }
 
+    public [LEXER_NAME]() throws IOException{
+        reInit();
+    }
+
+    public void setBuffer(char[] buffer){
+        this.buffer = buffer;
+        tokenBegin = bufpos = 0;
+        containsEscapes = false;
+        line++;
+        tokenBegin = -1;
+    }
+
+    public void reInit(){
+        bufsize        = Integer.MAX_VALUE;
+        endOf_UNUSED_Buffer = bufsize;
+        endOf_USED_Buffer = bufsize;
+        line           = 0;
+        prevCharIsCR   = false;
+        prevCharIsLF   = false;
+        tokenBegin     = -1;
+        maxUnusedBufferSize = bufsize;
+    }
+
     public void reInit(java.io.Reader stream) throws IOException{
         done();
         inputStream    = stream;
@@ -239,5 +262,5 @@
       bufsize += maxUnusedBufferSize;
       endOf_UNUSED_Buffer = bufsize;
       tokenBegin = 0;
-    }    
+    }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 5317fc2..a73a236 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -39,9 +39,13 @@
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.IAsterixPropertiesProvider;
 import org.apache.asterix.common.context.BaseOperationTracker;
+import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
 import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
+import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.runtime.GenericSocketFeedAdapterFactory;
+import org.apache.asterix.external.runtime.SocketClientAdapterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
@@ -76,12 +80,14 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactory;
@@ -316,19 +322,8 @@
     }
 
     private static void insertInitialAdapters(MetadataTransactionContext mdTxnCtx) throws Exception {
-        String[] builtInAdapterClassNames = new String[] {
-                "org.apache.asterix.external.adapter.factory.PullBasedAzureTwitterAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.NCFileSystemAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.HDFSAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.HiveAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.PushBasedTwitterAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.RSSFeedAdapterFactory",
-                "org.apache.asterix.external.adapter.factory.CNNFeedAdapterFactory",
-                "org.apache.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory",
-                "org.apache.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory",
-                "org.apache.asterix.tools.external.data.GenericSocketFeedAdapterFactory",
-                "org.apache.asterix.tools.external.data.SocketClientAdapterFactory" };
+        String[] builtInAdapterClassNames = new String[] { GenericAdapterFactory.class.getName(),
+                GenericSocketFeedAdapterFactory.class.getName(), SocketClientAdapterFactory.class.getName() };
         DatasourceAdapter adapter;
         for (String adapterClassName : builtInAdapterClassNames) {
             adapter = getAdapter(adapterClassName);
@@ -349,11 +344,9 @@
     }
 
     private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
-        String[] builtInCompactionPolicyClassNames = new String[] {
-                "org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory",
-                "org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory",
-                "org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory",
-                "org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory" };
+        String[] builtInCompactionPolicyClassNames = new String[] { ConstantMergePolicyFactory.class.getName(),
+                PrefixMergePolicyFactory.class.getName(), NoMergePolicyFactory.class.getName(),
+                CorrelatedPrefixMergePolicyFactory.class.getName() };
         CompactionPolicy compactionPolicy;
         for (String policyClassName : builtInCompactionPolicyClassNames) {
             compactionPolicy = getCompactionPolicyEntity(policyClassName);
@@ -362,7 +355,7 @@
     }
 
     private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws Exception {
-        String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getName();
+        String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getAlias();
         return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName),
                 adapterFactoryClassName, DatasourceAdapter.AdapterType.INTERNAL);
     }
@@ -378,8 +371,7 @@
         ClusterPartition metadataPartition = propertiesProvider.getMetadataProperties().getMetadataPartition();
         int metadataDeviceId = metadataPartition.getIODeviceNum();
         String metadataPartitionPath = SplitsAndConstraintsUtil.prepareStoragePartitionPath(
-                AsterixClusterProperties.INSTANCE.getStorageDirectoryName(),
-                metadataPartition.getPartitionId());
+                AsterixClusterProperties.INSTANCE.getStorageDirectoryName(), metadataPartition.getPartitionId());
         String resourceName = metadataPartitionPath + File.separator + index.getFileNameRelativePath();
         FileReference file = ioManager.getAbsoluteFileRef(metadataDeviceId, resourceName);
 
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 745f436..c9157df 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -23,7 +23,6 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -52,17 +51,17 @@
 import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
 import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.parse.IParseFileSplitsDecl;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory.SupportedOperation;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.indexing.dataflow.HDFSLookupAdapterFactory;
-import org.apache.asterix.external.indexing.operators.ExternalLoopkupOperatorDiscriptor;
+import org.apache.asterix.external.operators.ExternalBTreeSearchOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalLookupOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalRTreeSearchOperatorDescriptor;
+import org.apache.asterix.external.provider.AdapterFactoryProvider;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlLinearizeComparatorFactoryProvider;
@@ -100,8 +99,6 @@
 import org.apache.asterix.om.util.AsterixClusterProperties;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.base.AsterixTupleFilterFactory;
-import org.apache.asterix.runtime.external.ExternalBTreeSearchOperatorDescriptor;
-import org.apache.asterix.runtime.external.ExternalRTreeSearchOperatorDescriptor;
 import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
@@ -153,11 +150,8 @@
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
-import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 import org.apache.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
@@ -202,8 +196,6 @@
 
     private final AsterixStorageProperties storageProperties;
 
-    public static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
-
     public String getPropertyValue(String propertyName) {
         return config.get(propertyName);
     }
@@ -490,10 +482,7 @@
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(JobSpecification jobSpec,
             LoadableDataSource alds, IAdapterFactory adapterFactory, RecordDescriptor rDesc, boolean isPKAutoGenerated,
             List<List<String>> primaryKeys, ARecordType recType, int pkIndex) throws AlgebricksException {
-        if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ)
-                || adapterFactory.getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
-            throw new AlgebricksException(" External dataset adapter does not support read operation");
-        }
+
         ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, rDesc,
                 adapterFactory);
         AlgebricksPartitionConstraint constraint;
@@ -552,24 +541,9 @@
     private IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
             Map<String, String> configuration, IAType itemType, boolean isPKAutoGenerated,
             List<List<String>> primaryKeys) throws AlgebricksException {
-        IAdapterFactory adapterFactory;
-        DatasourceAdapter adapterEntity;
-        String adapterFactoryClassname;
         try {
-            adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
-                    adapterName);
-            if (adapterEntity != null) {
-                adapterFactoryClassname = adapterEntity.getClassname();
-                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-            } else {
-                adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
-                if (adapterFactoryClassname == null) {
-                    throw new AlgebricksException(" Unknown adapter :" + adapterName);
-                }
-                adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-            }
-
-            adapterFactory.configure(configuration, (ARecordType) itemType);
+            IAdapterFactory adapterFactory = AdapterFactoryProvider.getAdapterFactory(adapterName, configuration,
+                    (ARecordType) itemType);
 
             // check to see if dataset is indexed
             Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -602,11 +576,7 @@
             throw new AlgebricksException("Can only scan datasets of records.");
         }
 
-        if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ)
-                || adapterFactory.getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
-            throw new AlgebricksException(" External dataset adapter does not support read operation");
-        }
-
+        @SuppressWarnings("rawtypes")
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
@@ -623,33 +593,11 @@
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
     }
 
-    @SuppressWarnings("rawtypes")
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildScannerRuntime(JobSpecification jobSpec,
-            IAType itemType, IParseFileSplitsDecl decl, IDataFormat format) throws AlgebricksException {
-        if (itemType.getTypeTag() != ATypeTag.RECORD) {
-            throw new AlgebricksException("Can only scan datasets of records.");
-        }
-        ARecordType rt = (ARecordType) itemType;
-        ITupleParserFactory tupleParser = format.createTupleParser(rt, decl);
-        FileSplit[] splits = decl.getSplits();
-        IFileSplitProvider scannerSplitProvider = new ConstantFileSplitProvider(splits);
-        ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
-        RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-        IOperatorDescriptor scanner = new FileScanOperatorDescriptor(jobSpec, scannerSplitProvider, tupleParser,
-                scannerDesc);
-        String[] locs = new String[splits.length];
-        for (int i = 0; i < splits.length; i++) {
-            locs[i] = splits[i].getNodeName();
-        }
-        AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(locs);
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(scanner, apc);
-    }
-
-    public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory> buildFeedIntakeRuntime(
+    public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> buildFeedIntakeRuntime(
             JobSpecification jobSpec, PrimaryFeed primaryFeed, FeedPolicyAccessor policyAccessor) throws Exception {
-        Triple<IFeedAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
+        Triple<IAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
         factoryOutput = FeedUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx);
-        IFeedAdapterFactory adapterFactory = factoryOutput.first;
+        IAdapterFactory adapterFactory = factoryOutput.first;
         FeedIntakeOperatorDescriptor feedIngestor = null;
         switch (factoryOutput.third) {
             case INTERNAL:
@@ -665,7 +613,7 @@
         }
 
         AlgebricksPartitionConstraint partitionConstraint = adapterFactory.getPartitionConstraint();
-        return new Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory>(feedIngestor,
+        return new Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory>(feedIngestor,
                 partitionConstraint, adapterFactory);
     }
 
@@ -1515,7 +1463,7 @@
                     dataverseName, datasetName, indexName, dataset.getDatasetDetails().isTemp());
 
             // Generate Output Record format
-            ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
+            ISerializerDeserializer<?>[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
             ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
             ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
 
@@ -2102,7 +2050,6 @@
      * Calculate an estimate size of the bloom filter. Note that this is an
      * estimation which assumes that the data is going to be uniformly
      * distributed across all partitions.
-     *
      * @param dataset
      * @return Number of elements that will be used to create a bloom filter per
      *         dataset per partition
@@ -2147,24 +2094,6 @@
         return SplitsAndConstraintsUtil.splitsForDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName, temp);
     }
 
-    private static Map<String, String> initializeAdapterFactoryMapping() {
-        Map<String, String> adapterFactoryMapping = new HashMap<String, String>();
-        adapterFactoryMapping.put("org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter",
-                "org.apache.asterix.external.adapter.factory.NCFileSystemAdapterFactory");
-        adapterFactoryMapping.put("org.apache.asterix.external.dataset.adapter.HDFSAdapter",
-                "org.apache.asterix.external.adapter.factory.HDFSAdapterFactory");
-        adapterFactoryMapping.put("org.apache.asterix.external.dataset.adapter.PullBasedTwitterAdapter",
-                "org.apache.asterix.external.dataset.adapter.PullBasedTwitterAdapterFactory");
-        adapterFactoryMapping.put("org.apache.asterix.external.dataset.adapter.RSSFeedAdapter",
-                "org.apache.asterix.external.dataset.adapter..RSSFeedAdapterFactory");
-        adapterFactoryMapping.put("org.apache.asterix.external.dataset.adapter.CNNFeedAdapter",
-                "org.apache.asterix.external.dataset.adapter.CNNFeedAdapterFactory");
-        adapterFactoryMapping.put("org.apache.asterix.tools.external.data.RateControlledFileSystemBasedAdapter",
-                "org.apache.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory");
-
-        return adapterFactoryMapping;
-    }
-
     public DatasourceAdapter getAdapter(MetadataTransactionContext mdTxnCtx, String dataverseName, String adapterName)
             throws MetadataException {
         DatasourceAdapter adapter = null;
@@ -2232,35 +2161,6 @@
         return FormatUtils.getDefaultFormat();
     }
 
-    /**
-     * Add HDFS scheduler and the cluster location constraint into the scheduler
-     *
-     * @param properties
-     *            the original dataset properties
-     * @return a new map containing the original dataset properties and the
-     *         scheduler/locations
-     */
-    private static Map<String, Object> wrapProperties(Map<String, String> properties) {
-        Map<String, Object> wrappedProperties = new HashMap<String, Object>();
-        wrappedProperties.putAll(properties);
-        // wrappedProperties.put(SCHEDULER, hdfsScheduler);
-        // wrappedProperties.put(CLUSTER_LOCATIONS, getClusterLocations());
-        return wrappedProperties;
-    }
-
-    /**
-     * Adapt the original properties to a string-object map
-     *
-     * @param properties
-     *            the original properties
-     * @return the new stirng-object map
-     */
-    private static Map<String, Object> wrapPropertiesEmpty(Map<String, String> properties) {
-        Map<String, Object> wrappedProperties = new HashMap<String, Object>();
-        wrappedProperties.putAll(properties);
-        return wrappedProperties;
-    }
-
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForFilesIndex(
             String dataverseName, String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
         return SplitsAndConstraintsUtil.splitProviderAndPartitionConstraintsForFilesIndex(mdTxnCtx, dataverseName,
@@ -2284,67 +2184,54 @@
             IVariableTypeEnvironment typeEnv, List<LogicalVariable> outputVars, IOperatorSchema opSchema,
             JobGenContext context, AqlMetadataProvider metadataProvider, boolean retainNull)
                     throws AlgebricksException {
-        // Get data type
-        IAType itemType = null;
         try {
+            // Get data type
+            IAType itemType = null;
             itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
-        } catch (MetadataException e) {
-            e.printStackTrace();
-            throw new AlgebricksException("Unable to get item type from metadata " + e);
-        }
-        if (itemType.getTypeTag() != ATypeTag.RECORD) {
-            throw new AlgebricksException("Can only scan datasets of records.");
-        }
 
-        // Create the adapter factory <- right now there is only one. if there are more in the future, we can create a map->
-        ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        HDFSLookupAdapterFactory adapterFactory = new HDFSLookupAdapterFactory();
-        adapterFactory.configure(itemType, retainInput, ridIndexes, datasetDetails.getProperties(), retainNull);
+            // Create the adapter factory <- right now there is only one. if there are more in the future, we can create a map->
+            ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+            LookupAdapterFactory<?> adapterFactory = AdapterFactoryProvider.getAdapterFactory(
+                    datasetDetails.getProperties(), (ARecordType) itemType, ridIndexes, retainInput, retainNull,
+                    context.getNullWriterFactory());
 
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo;
-        try {
-            compactionInfo = DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
-        } catch (MetadataException e) {
-            throw new AlgebricksException(" Unabel to create merge policy factory for external dataset", e);
-        }
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo;
+            try {
+                compactionInfo = DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+            } catch (MetadataException e) {
+                throw new AlgebricksException(" Unabel to create merge policy factory for external dataset", e);
+            }
 
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        // Create the file index data flow helper
-        ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                compactionInfo.first, compactionInfo.second,
-                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
-                ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider), !temp);
+            boolean temp = datasetDetails.isTemp();
+            // Create the file index data flow helper
+            ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
+                    compactionInfo.first, compactionInfo.second,
+                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                    metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider), !temp);
 
-        // Create the out record descriptor, appContext and fileSplitProvider for the files index
-        RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-        try {
+            // Create the out record descriptor, appContext and fileSplitProvider for the files index
+            RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+            IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
             spPc = metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
-                    dataset.getDatasetName(), dataset.getDatasetName().
+                    dataset.getDatasetName(),
+                    dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
+            ISearchOperationCallbackFactory searchOpCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
+                    : new SecondaryIndexSearchOperationCallbackFactory();
+            // Create the operator
+            ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
+                    outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
+                    appContext.getStorageManagerInterface(), spPc.first, dataset.getDatasetId(),
+                    metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), searchOpCallbackFactory,
+                    retainNull, context.getNullWriterFactory());
 
-            concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
-        } catch (
-
-        Exception e)
-
-        {
+            // Return value
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, spPc.second);
+        } catch (Exception e) {
             throw new AlgebricksException(e);
         }
-
-        ISearchOperationCallbackFactory searchOpCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                : new SecondaryIndexSearchOperationCallbackFactory();
-        // Create the operator
-        ExternalLoopkupOperatorDiscriptor op = new ExternalLoopkupOperatorDiscriptor(jobSpec, adapterFactory,
-                outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
-                appContext.getStorageManagerInterface(), spPc.first, dataset.getDatasetId(),
-                metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), searchOpCallbackFactory,
-                retainNull, context.getNullWriterFactory());
-
-        // Return value
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, spPc.second);
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapter.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapter.java
deleted file mode 100644
index f4484cf..0000000
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapter.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.metadata.declared;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public class FieldExtractingAdapter implements IDatasourceAdapter {
-
-    private static final long serialVersionUID = 1L;
-
-    private final RecordDescriptor inRecDesc;
-
-    private final RecordDescriptor outRecDesc;
-
-    private final IDatasourceAdapter wrappedAdapter;
-
-    private final FieldExtractingPushRuntime fefw;
-
-    public FieldExtractingAdapter(IHyracksTaskContext ctx, RecordDescriptor inRecDesc, RecordDescriptor outRecDesc,
-            int[][] extractFields, ARecordType rType, IDatasourceAdapter wrappedAdapter) {
-        this.inRecDesc = inRecDesc;
-        this.outRecDesc = outRecDesc;
-        this.wrappedAdapter = wrappedAdapter;
-        fefw = new FieldExtractingPushRuntime(ctx, extractFields, rType);
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        fefw.setInputRecordDescriptor(0, inRecDesc);
-        fefw.setFrameWriter(0, writer, outRecDesc);
-        fefw.open();
-        try {
-            wrappedAdapter.start(partition, fefw);
-        } catch (Throwable t) {
-            fefw.fail();
-            throw t;
-        } finally {
-            fefw.close();
-        }
-    }
-
-    private static class FieldExtractingPushRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
-
-        private final IHyracksTaskContext ctx;
-
-        private final int[][] extractFields;
-
-        private final ARecordType rType;
-
-        private final int nullBitmapSize;
-
-        private final ArrayTupleBuilder tb;
-
-        public FieldExtractingPushRuntime(IHyracksTaskContext ctx, int[][] extractFields, ARecordType rType) {
-            this.ctx = ctx;
-            this.extractFields = extractFields;
-            this.rType = rType;
-            nullBitmapSize = ARecordType.computeNullBitmapSize(rType);
-            tb = new ArrayTupleBuilder(extractFields.length + 1);
-        }
-
-        @Override
-        public void open() throws HyracksDataException {
-            initAccessAppendRef(ctx);
-        }
-
-        @Override
-        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-            tAccess.reset(buffer);
-            for (int i = 0; i < tAccess.getTupleCount(); ++i) {
-                tb.reset();
-                tRef.reset(tAccess, i);
-                byte[] record = tRef.getFieldData(0);
-                int recStart = tRef.getFieldStart(0);
-                int recLength = tRef.getFieldLength(0);
-                for (int f = 0; f < extractFields.length; ++f) {
-                    try {
-                        byte[] subRecord = record;
-                        int subFStart = recStart;
-                        int subFOffset = 0;
-                        boolean isNull = false;
-                        IAType subFType = rType;
-                        int subFLen = recLength;
-                        int subBitMapSize = nullBitmapSize;
-                        byte[] subRecordTmp;
-
-                        for (int j = 0; j < extractFields[f].length; j++) {
-                            //Get offset for subfield
-                            subFOffset = ARecordSerializerDeserializer.getFieldOffsetById(subRecord, subFStart,
-                                    extractFields[f][j], subBitMapSize, ((ARecordType) subFType).isOpen());
-                            if (subFOffset == 0) {
-                                tb.getDataOutput().write(ATypeTag.NULL.serialize());
-                                isNull = true;
-                                break;
-                            } else {
-                                //Get type of subfield
-                                subFType = ((ARecordType) subFType).getFieldTypes()[extractFields[f][j]];
-                                try {
-                                    //Get length of subfield
-                                    subFLen = NonTaggedFormatUtil.getFieldValueLength(subRecord,
-                                            subFStart + subFOffset, subFType.getTypeTag(), false);
-
-                                    if (j < extractFields[f].length - 1) {
-                                        subRecordTmp = new byte[subFLen + 1];
-                                        subRecordTmp[0] = subFType.getTypeTag().serialize();
-                                        System.arraycopy(subRecord, subFStart + subFOffset, subRecordTmp, 1, subFLen);
-                                        subRecord = subRecordTmp;
-                                        subFStart = 0;
-                                        subBitMapSize = ARecordType.computeNullBitmapSize((ARecordType) subFType);
-                                    }
-
-                                } catch (AsterixException e) {
-                                    throw new HyracksDataException(e);
-                                }
-                            }
-                        }
-
-                        if (!isNull) {
-                            tb.getDataOutput().write(subFType.getTypeTag().serialize());
-                            tb.getDataOutput().write(subRecord, subFStart + subFOffset, subFLen);
-                        }
-
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
-                    }
-                    tb.addFieldEndOffset();
-                }
-                tb.addField(record, recStart, tRef.getFieldLength(0));
-                appendToFrameFromTupleBuilder(tb);
-            }
-        }
-
-        @Override
-        public void close() throws HyracksDataException {
-            flushIfNotFailed();
-        }
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapterFactory.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapterFactory.java
deleted file mode 100644
index 989e4a3..0000000
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FieldExtractingAdapterFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.metadata.declared;
-
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory.SupportedOperation;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-
-public class FieldExtractingAdapterFactory implements IAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final IAdapterFactory wrappedAdapterFactory;
-
-    private final RecordDescriptor inRecDesc;
-
-    private final RecordDescriptor outRecDesc;
-
-    private final int[][] extractFields;
-
-    private final ARecordType rType;
-
-    public FieldExtractingAdapterFactory(IAdapterFactory wrappedAdapterFactory, RecordDescriptor inRecDesc,
-            RecordDescriptor outRecDesc, int[][] extractFields, ARecordType rType) {
-        this.wrappedAdapterFactory = wrappedAdapterFactory;
-        this.inRecDesc = inRecDesc;
-        this.outRecDesc = outRecDesc;
-        this.extractFields = extractFields;
-        this.rType = rType;
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return wrappedAdapterFactory.getSupportedOperations();
-    }
-
-    @Override
-    public String getName() {
-        return "FieldExtractingAdapter[ " + wrappedAdapterFactory.getName() + " ]";
-    }
-
-  
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return wrappedAdapterFactory.getPartitionConstraint();
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        IDatasourceAdapter wrappedAdapter = wrappedAdapterFactory.createAdapter(ctx, partition);
-        return new FieldExtractingAdapter(ctx, inRecDesc, outRecDesc, extractFields, rType, wrappedAdapter);
-    }
-    
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        wrappedAdapterFactory.configure(configuration, outputType);
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return wrappedAdapterFactory.getAdapterOutputType();
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapter.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapter.java
deleted file mode 100644
index e0c5fc0..0000000
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapter.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.metadata.declared;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.om.base.AMutableUUID;
-import org.apache.asterix.om.base.AUUID;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.PointableAllocator;
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public class PKGeneratingAdapter implements IDatasourceAdapter {
-
-    private static final long serialVersionUID = 1L;
-    private final RecordDescriptor inRecDesc;
-    private final RecordDescriptor outRecDesc;
-    private final IDatasourceAdapter wrappedAdapter;
-    private final PKGeneratingPushRuntime pkRuntime;
-    private final int pkIndex;
-
-    public PKGeneratingAdapter(IHyracksTaskContext ctx, RecordDescriptor inRecDesc, RecordDescriptor outRecDesc,
-            ARecordType inRecType, ARecordType outRecType, IDatasourceAdapter wrappedAdapter, int pkIndex) {
-        this.inRecDesc = inRecDesc;
-        this.outRecDesc = outRecDesc;
-        this.wrappedAdapter = wrappedAdapter;
-        this.pkRuntime = new PKGeneratingPushRuntime(ctx, inRecType, outRecType);
-        this.pkIndex = pkIndex;
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-        pkRuntime.setInputRecordDescriptor(0, inRecDesc);
-        pkRuntime.setFrameWriter(0, writer, outRecDesc);
-        pkRuntime.open();
-        try {
-            wrappedAdapter.start(partition, pkRuntime);
-        } catch (Throwable t) {
-            pkRuntime.fail();
-            throw t;
-        } finally {
-            pkRuntime.close();
-        }
-    }
-
-    private class PKGeneratingPushRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
-        private final IHyracksTaskContext ctx;
-        private final ARecordType outRecType;
-        private final ArrayTupleBuilder tb;
-        private final AMutableUUID aUUID = new AMutableUUID(0, 0);
-        private final byte AUUIDTag = ATypeTag.UUID.serialize();
-        private final byte[] serializedUUID = new byte[16];
-        private final PointableAllocator pa = new PointableAllocator();
-        private final ARecordVisitablePointable recordPointable;
-        private final IAType[] outClosedTypes;
-
-        private final RecordBuilder recBuilder;
-
-        public PKGeneratingPushRuntime(IHyracksTaskContext ctx, ARecordType inRecType, ARecordType outRecType) {
-            this.ctx = ctx;
-            this.outRecType = outRecType;
-            this.tb = new ArrayTupleBuilder(2);
-            this.recBuilder = new RecordBuilder();
-            this.recordPointable = (ARecordVisitablePointable) pa.allocateRecordValue(inRecType);
-            this.outClosedTypes = outRecType.getFieldTypes();
-        }
-
-        /*
-         * We write this method in low level instead of using pre-existing libraries since this will be called for each record and to avoid 
-         * size validation
-         */
-        private void serializeUUID(AUUID aUUID, byte[] serializedUUID) {
-            long v = aUUID.getLeastSignificantBits();
-            serializedUUID[0] = (byte) (v >>> 56);
-            serializedUUID[1] = (byte) (v >>> 48);
-            serializedUUID[2] = (byte) (v >>> 40);
-            serializedUUID[3] = (byte) (v >>> 32);
-            serializedUUID[4] = (byte) (v >>> 24);
-            serializedUUID[5] = (byte) (v >>> 16);
-            serializedUUID[6] = (byte) (v >>> 8);
-            serializedUUID[7] = (byte) (v >>> 0);
-            v = aUUID.getMostSignificantBits();
-            serializedUUID[8] = (byte) (v >>> 56);
-            serializedUUID[9] = (byte) (v >>> 48);
-            serializedUUID[10] = (byte) (v >>> 40);
-            serializedUUID[11] = (byte) (v >>> 32);
-            serializedUUID[12] = (byte) (v >>> 24);
-            serializedUUID[13] = (byte) (v >>> 16);
-            serializedUUID[14] = (byte) (v >>> 8);
-            serializedUUID[15] = (byte) (v >>> 0);
-        }
-
-        @Override
-        public void open() throws HyracksDataException {
-            initAccessAppendRef(ctx);
-            recBuilder.reset(outRecType);
-            recBuilder.init();
-        }
-
-        @Override
-        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-            try {
-                tAccess.reset(buffer);
-                for (int i = 0; i < tAccess.getTupleCount(); ++i) {
-                    tb.reset();
-                    tRef.reset(tAccess, i);
-
-                    // We need to do the following:
-                    // 1. generate a UUID
-                    // 2. fill in the first field with the UUID
-                    aUUID.nextUUID();
-                    tb.getDataOutput().writeByte(AUUIDTag);
-                    serializeUUID(aUUID, serializedUUID);
-                    tb.getDataOutput().write(serializedUUID);
-                    tb.addFieldEndOffset();
-                    // 3. fill in the second field with the record after adding to it the UUID
-                    recordPointable.set(tRef.getFieldData(0), tRef.getFieldStart(0), tRef.getFieldLength(0));
-                    // Start by closed fields
-                    int inIndex = 0;
-                    for (int f = 0; f < outClosedTypes.length; f++) {
-                        if (f == pkIndex) {
-                            recBuilder.addField(f, serializedUUID);
-                        } else {
-                            recBuilder.addField(f, recordPointable.getFieldValues().get(inIndex));
-                            inIndex++;
-                        }
-                    }
-
-                    // Add open fields
-                    if (outRecType.isOpen()) {
-                        List<IVisitablePointable> fp = recordPointable.getFieldNames();
-                        if (fp.size() >= outClosedTypes.length) {
-                            int index = outClosedTypes.length - 1;
-                            while (index < fp.size()) {
-                                recBuilder.addField(fp.get(index), recordPointable.getFieldValues().get(index));
-                                index++;
-                            }
-                        }
-                    }
-                    //write the record
-                    recBuilder.write(tb.getDataOutput(), true);
-                    tb.addFieldEndOffset();
-                    appendToFrameFromTupleBuilder(tb);
-                }
-            } catch (Exception e) {
-                throw new HyracksDataException("Error in the auto id generation and merge of the record", e);
-            }
-        }
-
-        @Override
-        public void close() throws HyracksDataException {
-            flushIfNotFailed();
-        }
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapterFactory.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapterFactory.java
deleted file mode 100644
index 2930662..0000000
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/PKGeneratingAdapterFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.metadata.declared;
-
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-
-/**
- *
- * @author alamouda
- * This wrapper adapter factory is an adapter that is used when loading data into a dataset with a UUID primary key. The factory creates an adapter that
- * generates UUIDs and merge them into parsed records
- */
-public class PKGeneratingAdapterFactory implements IAdapterFactory {
-
-    private static final long serialVersionUID = 1L;
-    private final IAdapterFactory wrappedAdapterFactory;
-    private final RecordDescriptor inRecDesc;
-    private final RecordDescriptor outRecDesc;
-    private final ARecordType inRecType;
-    private final ARecordType outRecType;
-    private final int pkIndex;
-
-    public PKGeneratingAdapterFactory(IAdapterFactory wrappedAdapterFactory, RecordDescriptor inRecDesc,
-            RecordDescriptor outRecDesc, ARecordType inRecType, ARecordType outRecType, int pkIndex) {
-        this.wrappedAdapterFactory = wrappedAdapterFactory;
-        this.inRecDesc = inRecDesc;
-        this.outRecDesc = outRecDesc;
-        this.inRecType = inRecType;
-        this.outRecType = outRecType;
-        this.pkIndex = pkIndex;
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return wrappedAdapterFactory.getSupportedOperations();
-    }
-
-    @Override
-    public String getName() {
-        return "PKGeneratingAdapter[ " + wrappedAdapterFactory.getName() + " ]";
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return wrappedAdapterFactory.getPartitionConstraint();
-    }
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        IDatasourceAdapter wrappedAdapter = wrappedAdapterFactory.createAdapter(ctx, partition);
-        return new PKGeneratingAdapter(ctx, inRecDesc, outRecDesc, inRecType, outRecType, wrappedAdapter, pkIndex);
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        wrappedAdapterFactory.configure(configuration, outputType);        
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return wrappedAdapterFactory.getAdapterOutputType();
-    }
-}
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractDatasourceAdapter.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractDatasourceAdapter.java
index 7f6a567..d65468e 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractDatasourceAdapter.java
@@ -20,7 +20,7 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -29,7 +29,7 @@
  * Represents the base class that is required to be extended by every
  * implementation of the IDatasourceAdapter interface.
  */
-public abstract class AbstractDatasourceAdapter implements IDatasourceAdapter {
+public abstract class AbstractDatasourceAdapter implements IDataSourceAdapter {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
index a87d2fb..c231ad9 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
@@ -18,11 +18,11 @@
  */
 package org.apache.asterix.metadata.feeds;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.external.feeds.FeedPolicyEnforcer;
 
 
-public abstract class AbstractFeedDatasourceAdapter implements IDatasourceAdapter {
+public abstract class AbstractFeedDatasourceAdapter implements IDataSourceAdapter {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterExecutor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterExecutor.java
index 604ef79..6c2f14c 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterExecutor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterExecutor.java
@@ -24,7 +24,7 @@
 import org.apache.asterix.common.feeds.DistributeFeedFrameWriter;
 import org.apache.asterix.common.feeds.api.IAdapterRuntimeManager;
 import org.apache.asterix.common.feeds.api.IAdapterRuntimeManager.State;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 
 public class AdapterExecutor implements Runnable {
 
@@ -32,11 +32,11 @@
 
     private final DistributeFeedFrameWriter writer;
 
-    private final IFeedAdapter adapter;
+    private final IDataSourceAdapter adapter;
 
     private final IAdapterRuntimeManager adapterManager;
 
-    public AdapterExecutor(int partition, DistributeFeedFrameWriter writer, IFeedAdapter adapter,
+    public AdapterExecutor(int partition, DistributeFeedFrameWriter writer, IDataSourceAdapter adapter,
             IAdapterRuntimeManager adapterManager) {
         this.writer = writer;
         this.adapter = adapter;
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterRuntimeManager.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterRuntimeManager.java
index 0e8ce1a..aacb3da 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterRuntimeManager.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/AdapterRuntimeManager.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.common.feeds.FeedId;
 import org.apache.asterix.common.feeds.IngestionRuntime;
 import org.apache.asterix.common.feeds.api.IAdapterRuntimeManager;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
 
 public class AdapterRuntimeManager implements IAdapterRuntimeManager {
@@ -36,7 +36,7 @@
 
     private final FeedId feedId;
 
-    private final IFeedAdapter feedAdapter;
+    private final IDataSourceAdapter feedAdapter;
 
     private final IIntakeProgressTracker tracker;
 
@@ -50,7 +50,7 @@
 
     private State state;
 
-    public AdapterRuntimeManager(FeedId feedId, IFeedAdapter feedAdapter, IIntakeProgressTracker tracker,
+    public AdapterRuntimeManager(FeedId feedId, IDataSourceAdapter feedAdapter, IIntakeProgressTracker tracker,
             DistributeFeedFrameWriter writer, int partition) {
         this.feedId = feedId;
         this.feedAdapter = feedAdapter;
@@ -91,7 +91,8 @@
         return feedId + "[" + partition + "]";
     }
 
-    public IFeedAdapter getFeedAdapter() {
+    @Override
+    public IDataSourceAdapter getFeedAdapter() {
         return feedAdapter;
     }
 
@@ -99,10 +100,12 @@
         return tracker;
     }
 
+    @Override
     public synchronized State getState() {
         return state;
     }
 
+    @Override
     public synchronized void setState(State state) {
         this.state = state;
     }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
index ba985bc..fee99d8 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.metadata.feeds;
 
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -55,7 +55,7 @@
 
             @Override
             public void initialize() throws HyracksDataException {
-                IDatasourceAdapter adapter = null;
+                IDataSourceAdapter adapter = null;
                 try {
                     writer.open();
                     adapter = adapterFactory.createAdapter(ctx, partition);
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
index 486f45b..54c9af5 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -27,9 +27,9 @@
 import org.apache.asterix.common.feeds.IngestionRuntime;
 import org.apache.asterix.common.feeds.SubscribableFeedRuntimeId;
 import org.apache.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
-import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.common.feeds.api.IFeedSubscriptionManager;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.metadata.entities.PrimaryFeed;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -55,7 +55,7 @@
     private final FeedPolicyAccessor policyAccessor;
 
     /** The adaptor factory that is used to create an instance of the feed adaptor **/
-    private IFeedAdapterFactory adaptorFactory;
+    private IAdapterFactory adaptorFactory;
 
     /** The library that contains the adapter in use. **/
     private String adaptorLibraryName;
@@ -71,8 +71,8 @@
 
     private ARecordType adapterOutputType;
 
-    public FeedIntakeOperatorDescriptor(JobSpecification spec, PrimaryFeed primaryFeed,
-            IFeedAdapterFactory adapterFactory, ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor) {
+    public FeedIntakeOperatorDescriptor(JobSpecification spec, PrimaryFeed primaryFeed, IAdapterFactory adapterFactory,
+            ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor) {
         super(spec, 0, 1);
         this.feedId = new FeedId(primaryFeed.getDataverseName(), primaryFeed.getFeedName());
         this.adaptorFactory = adapterFactory;
@@ -113,16 +113,16 @@
                 policyAccessor);
     }
 
-    private IFeedAdapterFactory createExtenralAdapterFactory(IHyracksTaskContext ctx, int partition) throws Exception {
-        IFeedAdapterFactory adapterFactory = null;
+    private IAdapterFactory createExtenralAdapterFactory(IHyracksTaskContext ctx, int partition) throws Exception {
+        IAdapterFactory adapterFactory = null;
         ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(feedId.getDataverse(),
                 adaptorLibraryName);
         if (classLoader != null) {
-            adapterFactory = ((IFeedAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance()));
+            adapterFactory = ((IAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance()));
             adapterFactory.configure(adaptorConfiguration, adapterOutputType);
         } else {
-            String message = "Unable to create adapter as class loader not configured for library "
-                    + adaptorLibraryName + " in dataverse " + feedId.getDataverse();
+            String message = "Unable to create adapter as class loader not configured for library " + adaptorLibraryName
+                    + " in dataverse " + feedId.getDataverse();
             LOGGER.severe(message);
             throw new IllegalArgumentException(message);
         }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
index 404d37c..5085087 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -33,13 +33,13 @@
 import org.apache.asterix.common.feeds.SubscribableFeedRuntimeId;
 import org.apache.asterix.common.feeds.api.IAdapterRuntimeManager;
 import org.apache.asterix.common.feeds.api.IAdapterRuntimeManager.State;
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
+import org.apache.asterix.common.feeds.api.IDataSourceAdapter;
 import org.apache.asterix.common.feeds.api.IFeedManager;
 import org.apache.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
 import org.apache.asterix.common.feeds.api.IFeedSubscriptionManager;
 import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
 import org.apache.asterix.common.feeds.api.ISubscriberRuntime;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -59,14 +59,14 @@
     private final IFeedSubscriptionManager feedSubscriptionManager;
     private final IFeedManager feedManager;
     private final IHyracksTaskContext ctx;
-    private final IFeedAdapterFactory adapterFactory;
+    private final IAdapterFactory adapterFactory;
 
     private IngestionRuntime ingestionRuntime;
-    private IFeedAdapter adapter;
+    private IDataSourceAdapter adapter;
     private IIntakeProgressTracker tracker;
     private DistributeFeedFrameWriter feedFrameWriter;
 
-    public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IFeedAdapterFactory adapterFactory,
+    public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IAdapterFactory adapterFactory,
             int partition, IngestionRuntime ingestionRuntime, FeedPolicyAccessor policyAccessor) {
         this.ctx = ctx;
         this.feedId = feedId;
@@ -85,12 +85,13 @@
         try {
             if (ingestionRuntime == null) {
                 try {
-                    adapter = (IFeedAdapter) adapterFactory.createAdapter(ctx, partition);
-                    if (adapterFactory.isRecordTrackingEnabled()) {
-                        tracker = adapterFactory.createIntakeProgressTracker();
-                    }
+                    adapter = adapterFactory.createAdapter(ctx, partition);
+                    //TODO: Fix record tracking
+                    //                    if (adapterFactory.isRecordTrackingEnabled()) {
+                    //                        tracker = adapterFactory.createIntakeProgressTracker();
+                    //                    }
                 } catch (Exception e) {
-                    LOGGER.severe("Unable to create adapter : " + adapterFactory.getName() + "[" + partition + "]"
+                    LOGGER.severe("Unable to create adapter : " + adapterFactory.getAlias() + "[" + partition + "]"
                             + " Exception " + e);
                     throw new HyracksDataException(e);
                 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedUtil.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedUtil.java
index 72b7c15..5ed2876 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedUtil.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedUtil.java
@@ -38,14 +38,14 @@
 import org.apache.asterix.common.feeds.FeedRuntimeId;
 import org.apache.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.library.ExternalLibraryManager;
+import org.apache.asterix.external.provider.AdapterFactoryProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.bootstrap.MetadataConstants;
-import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.DatasourceAdapter.AdapterType;
@@ -466,18 +466,21 @@
         return preProcessingRequired;
     }
 
-    public static Triple<IFeedAdapterFactory, ARecordType, AdapterType> getPrimaryFeedFactoryAndOutput(PrimaryFeed feed,
+    public static Triple<IAdapterFactory, ARecordType, AdapterType> getPrimaryFeedFactoryAndOutput(PrimaryFeed feed,
             FeedPolicyAccessor policyAccessor, MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
 
         String adapterName = null;
         DatasourceAdapter adapterEntity = null;
         String adapterFactoryClassname = null;
-        IFeedAdapterFactory adapterFactory = null;
+        IAdapterFactory adapterFactory = null;
         ARecordType adapterOutputType = null;
-        Triple<IFeedAdapterFactory, ARecordType, AdapterType> feedProps = null;
+        Triple<IAdapterFactory, ARecordType, AdapterType> feedProps = null;
         AdapterType adapterType = null;
         try {
             adapterName = feed.getAdaptorName();
+            Map<String, String> configuration = feed.getAdaptorConfiguration();
+            configuration.putAll(policyAccessor.getFeedPolicy());
+            adapterOutputType = getOutputType(feed, configuration);
             adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
                     adapterName);
             if (adapterEntity == null) {
@@ -488,30 +491,24 @@
                 adapterFactoryClassname = adapterEntity.getClassname();
                 switch (adapterType) {
                     case INTERNAL:
-                        adapterFactory = (IFeedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                        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 = (IFeedAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
+                        adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
                         break;
                 }
+                adapterFactory.configure(configuration, adapterOutputType);
             } else {
-                adapterFactoryClassname = AqlMetadataProvider.adapterFactoryMapping.get(adapterName);
-                if (adapterFactoryClassname == null) {
-                    adapterFactoryClassname = adapterName;
-                }
-                adapterFactory = (IFeedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+                configuration.put(ExternalDataConstants.KEY_DATAVERSE, feed.getDataverseName());
+                adapterFactory = AdapterFactoryProvider.getAdapterFactory(adapterName, configuration,
+                        adapterOutputType);
                 adapterType = AdapterType.INTERNAL;
             }
-
-            Map<String, String> configuration = feed.getAdaptorConfiguration();
-            configuration.putAll(policyAccessor.getFeedPolicy());
-            adapterOutputType = getOutputType(feed, configuration);
-            adapterFactory.configure(configuration, adapterOutputType);
-            feedProps = new Triple<IFeedAdapterFactory, ARecordType, AdapterType>(adapterFactory, adapterOutputType,
+            feedProps = new Triple<IAdapterFactory, ARecordType, AdapterType>(adapterFactory, adapterOutputType,
                     adapterType);
         } catch (Exception e) {
             e.printStackTrace();
@@ -522,7 +519,7 @@
 
     private static ARecordType getOutputType(PrimaryFeed feed, Map<String, String> configuration) throws Exception {
         ARecordType outputType = null;
-        String fqOutputType = configuration.get(IAdapterFactory.KEY_TYPE_NAME);
+        String fqOutputType = configuration.get(ExternalDataConstants.KEY_TYPE_NAME);
 
         if (fqOutputType == null) {
             throw new IllegalArgumentException("No output type specified");
@@ -538,7 +535,8 @@
             dataverseName = dataverseAndType[0];
             datatypeName = dataverseAndType[1];
         } else
-            throw new IllegalArgumentException("Invalid value for the parameter " + IAdapterFactory.KEY_TYPE_NAME);
+            throw new IllegalArgumentException(
+                    "Invalid value for the parameter " + ExternalDataConstants.KEY_TYPE_NAME);
 
         MetadataTransactionContext ctx = null;
         MetadataManager.INSTANCE.acquireReadLatch();
@@ -569,7 +567,7 @@
         Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feed.getDataverseName(), primaryFeedName);
         FunctionSignature appliedFunction = primaryFeed.getAppliedFunction();
         if (appliedFunction == null) {
-            Triple<IFeedAdapterFactory, ARecordType, AdapterType> result = getPrimaryFeedFactoryAndOutput(
+            Triple<IAdapterFactory, ARecordType, AdapterType> result = getPrimaryFeedFactoryAndOutput(
                     (PrimaryFeed) primaryFeed, policyAccessor, mdTxnCtx);
             outputType = result.second.getTypeName();
         } else {
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ITypedAdapterFactory.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ITypedAdapterFactory.java
index 7a26560..f35c21f 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ITypedAdapterFactory.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/ITypedAdapterFactory.java
@@ -20,7 +20,7 @@
 
 import java.util.Map;
 
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.om.types.ARecordType;
 
 public interface ITypedAdapterFactory extends IAdapterFactory {
diff --git a/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 8dcbb77..4872ede 100644
--- a/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -20,7 +20,6 @@
 
 import java.util.List;
 
-import org.apache.asterix.common.parse.IParseFileSplitsDecl;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
@@ -43,7 +42,6 @@
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
 public interface IDataFormat {
     public void registerRuntimeFunctions() throws AlgebricksException;
@@ -74,12 +72,8 @@
     public Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
             ARecordType recType, List<String> fldName) throws AlgebricksException;
 
-    public ICopyEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName, int recordColumn)
-            throws AlgebricksException;
-
-    public ITupleParserFactory createTupleParser(ARecordType recType, IParseFileSplitsDecl decl);
-
-    public ITupleParserFactory createTupleParser(ARecordType recType, boolean isDelimited, char delimiter, char quote, boolean hasHeader);
+    public ICopyEvaluatorFactory getFieldAccessEvaluatorFactory(ARecordType recType, List<String> fldName,
+            int recordColumn) throws AlgebricksException;
 
     public IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment typeEnvironment)
             throws AlgebricksException;
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixRuntimeUtil.java b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixRuntimeUtil.java
index dcc4891..51c3802 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixRuntimeUtil.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixRuntimeUtil.java
@@ -52,4 +52,8 @@
         AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext().getIPAddressNodeMap(map);
         return map;
     }
+
+    public static void getNodeControllerMap(Map<InetAddress, Set<String>> map) throws Exception {
+        AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext().getIPAddressNodeMap(map);
+    }
 }
diff --git a/asterix-runtime/pom.xml b/asterix-runtime/pom.xml
index 2713dd9..4f2d0b2 100644
--- a/asterix-runtime/pom.xml
+++ b/asterix-runtime/pom.xml
@@ -17,171 +17,63 @@
  ! under the License.
  !-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<artifactId>asterix</artifactId>
-		<groupId>org.apache.asterix</groupId>
-		<version>0.8.8-SNAPSHOT</version>
-	</parent>
-	<artifactId>asterix-runtime</artifactId>
-
-  <licenses>
-    <license>
-      <name>Apache License, Version 2.0</name>
-      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-      <distribution>repo</distribution>
-      <comments>A business-friendly OSS license</comments>
-    </license>
-  </licenses>
-
-
-	<build>
-		<plugins>
-			<plugin>
-	        	<groupId>org.apache.asterix</groupId>
-	        	<artifactId>lexer-generator-maven-plugin</artifactId>
-	        	<version>0.8.8-SNAPSHOT</version>
-	        	<configuration>
-	        	  	<grammarFile>src/main/resources/adm.grammar</grammarFile>
-	        	  	<outputDir>${project.build.directory}/generated-sources/org/apache/asterix/runtime/operators/file/adm</outputDir>
-	        	</configuration>
-	        	<executions>
-		          	<execution>
-		          		<id>generate-lexer</id>
-            			<phase>generate-sources</phase>
-			            <goals>
-	              			<goal>generate-lexer</goal>
-	            		</goals>
-	          		</execution>
-	        	</executions>
-	      	</plugin>
-	      	 <plugin>
-			    <groupId>org.codehaus.mojo</groupId>
-			    <artifactId>build-helper-maven-plugin</artifactId>
-			    <version>1.9</version>
-			    <executions>
-			        <execution>
-			            <id>add-source</id>
-			            <phase>generate-sources</phase>
-			            <goals>
-			                <goal>add-source</goal>
-			            </goals>
-			            <configuration>
-			                <sources>
-			                    <source>${project.build.directory}/generated-sources/</source>
-			                </sources>
-			            </configuration>
-			        </execution>
-			    </executions>
- 			</plugin>
-	    </plugins>
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>
-											org.apache.asterix
-										</groupId>
-										<artifactId>
-											lexer-generator-maven-plugin
-										</artifactId>
-										<versionRange>
-											[0.1,)
-										</versionRange>
-										<goals>
-											<goal>generate-lexer</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<execute>
-											<runOnIncremental>false</runOnIncremental>
-										</execute>
-									</action>
-								</pluginExecution>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>
-											org.codehaus.mojo
-										</groupId>
-										<artifactId>
-											build-helper-maven-plugin
-										</artifactId>
-										<versionRange>
-											[1.7,)
-										</versionRange>
-										<goals>
-											<goal>add-source</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore />
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-	</build>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.asterix</groupId>
-			<artifactId>asterix-om</artifactId>
-			<version>0.8.8-SNAPSHOT</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.asterix</groupId>
-			<artifactId>asterix-fuzzyjoin</artifactId>
-			<version>0.8.8-SNAPSHOT</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hyracks</groupId>
-			<artifactId>hyracks-storage-am-btree</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.asterix</groupId>
-			<artifactId>asterix-transactions</artifactId>
-			<version>0.8.8-SNAPSHOT</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.twitter4j</groupId>
-			<artifactId>twitter4j-core</artifactId>
-			<version>[4.0,)</version>
-		</dependency>
-		<dependency>
-			<groupId>org.twitter4j</groupId>
-			<artifactId>twitter4j-stream</artifactId>
-			<version>[4.0,)</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-client</artifactId>
-			<type>jar</type>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-		         <groupId>org.apache.hyracks</groupId>
-		         <artifactId>hyracks-api</artifactId>
-	        </dependency>
-	    <dependency>
-            <groupId>com.e-movimento.tinytools</groupId>
-            <artifactId>privilegedaccessor</artifactId>
-            <version>1.2.2</version>
-            <scope>test</scope>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>asterix</artifactId>
+        <groupId>org.apache.asterix</groupId>
+        <version>0.8.8-SNAPSHOT</version>
+    </parent>
+    <artifactId>asterix-runtime</artifactId>
+    <licenses>
+        <license>
+            <name>Apache License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+            <comments>A business-friendly OSS license</comments>
+        </license>
+    </licenses>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-om</artifactId>
+            <version>0.8.8-SNAPSHOT</version>
+            <scope>compile</scope>
         </dependency>
-	</dependencies>
-
-</project>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-fuzzyjoin</artifactId>
+            <version>0.8.8-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-storage-am-btree</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-transactions</artifactId>
+            <version>0.8.8-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.twitter4j</groupId>
+            <artifactId>twitter4j-core</artifactId>
+            <version>[4.0,)</version>
+        </dependency>
+        <dependency>
+            <groupId>org.twitter4j</groupId>
+            <artifactId>twitter4j-stream</artifactId>
+            <version>[4.0,)</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <type>jar</type>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-api</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index ef7a6c8..309f677 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -18,11 +18,19 @@
  */
 package org.apache.asterix.runtime.formats;
 
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.AsterixRuntimeException;
-import org.apache.asterix.common.parse.IParseFileSplitsDecl;
 import org.apache.asterix.dataflow.data.nontagged.AqlNullWriterFactory;
 import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.formats.nontagged.AqlADMPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlBinaryHashFunctionFactoryProvider;
@@ -33,7 +41,6 @@
 import org.apache.asterix.formats.nontagged.AqlLosslessJSONPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlNormalizedKeyComputerFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlPredicateEvaluatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlADMPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.AqlTypeTraitProvider;
 import org.apache.asterix.om.base.ABoolean;
@@ -166,7 +173,6 @@
 import org.apache.asterix.runtime.evaluators.functions.CastListDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CountHashedGramTokensDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CountHashedWordTokensDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CreateCircleDescriptor;
@@ -182,7 +188,6 @@
 import org.apache.asterix.runtime.evaluators.functions.EditDistanceListIsFilterable;
 import org.apache.asterix.runtime.evaluators.functions.EditDistanceStringIsFilterable;
 import org.apache.asterix.runtime.evaluators.functions.EmbedTypeDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FlowRecordDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FuzzyEqDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
@@ -193,7 +198,6 @@
 import org.apache.asterix.runtime.evaluators.functions.IsNullDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.IsSystemNullDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.LenDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringLikeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NotNullDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
@@ -223,16 +227,19 @@
 import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringConcatDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringEqualDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringJoinDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringLengthDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringLikeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringLowerCaseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringMatchesDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringMatchesWithFlagDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithFlagsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringUpperCaseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.Substring2Descriptor;
@@ -305,12 +312,12 @@
 import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
 import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -349,17 +356,6 @@
 import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 
 public class NonTaggedDataFormat implements IDataFormat {
 
@@ -386,6 +382,7 @@
     public NonTaggedDataFormat() {
     }
 
+    @Override
     public void registerRuntimeFunctions() throws AlgebricksException {
 
         if (registered) {
@@ -739,13 +736,13 @@
                     fieldFound = true;
                     try {
                         AInt32 ai = new AInt32(i);
-                        AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(ai.getType()).serialize(
-                                ai, dos);
+                        AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(ai.getType()).serialize(ai,
+                                dos);
                     } catch (HyracksDataException e) {
                         throw new AlgebricksException(e);
                     }
-                    ICopyEvaluatorFactory fldIndexEvalFactory = new ConstantEvalFactory(Arrays.copyOf(
-                            abvs.getByteArray(), abvs.getLength()));
+                    ICopyEvaluatorFactory fldIndexEvalFactory = new ConstantEvalFactory(
+                            Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
 
                     evalFactory = new FieldAccessByIndexEvalFactory(recordEvalFactory, fldIndexEvalFactory, recType);
                     return evalFactory;
@@ -775,8 +772,8 @@
             if (fldName.size() > 1) {
                 evalFactory = new FieldAccessNestedEvalFactory(recordEvalFactory, recType, fldName);
             } else {
-                evalFactory = FieldAccessByNameDescriptor.FACTORY.createFunctionDescriptor().createEvaluatorFactory(
-                        factories);
+                evalFactory = FieldAccessByNameDescriptor.FACTORY.createFunctionDescriptor()
+                        .createEvaluatorFactory(factories);
             }
             return evalFactory;
         } else
@@ -800,8 +797,8 @@
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
-        ICopyEvaluatorFactory dimensionEvalFactory = new ConstantEvalFactory(Arrays.copyOf(abvs1.getByteArray(),
-                abvs1.getLength()));
+        ICopyEvaluatorFactory dimensionEvalFactory = new ConstantEvalFactory(
+                Arrays.copyOf(abvs1.getByteArray(), abvs1.getLength()));
 
         for (int i = 0; i < numOfFields; i++) {
             ArrayBackedValueStorage abvs2 = new ArrayBackedValueStorage();
@@ -812,8 +809,8 @@
             } catch (HyracksDataException e) {
                 throw new AlgebricksException(e);
             }
-            ICopyEvaluatorFactory coordinateEvalFactory = new ConstantEvalFactory(Arrays.copyOf(abvs2.getByteArray(),
-                    abvs2.getLength()));
+            ICopyEvaluatorFactory coordinateEvalFactory = new ConstantEvalFactory(
+                    Arrays.copyOf(abvs2.getByteArray(), abvs2.getLength()));
 
             evalFactories[i] = new CreateMBREvalFactory(evalFactory, dimensionEvalFactory, coordinateEvalFactory);
         }
@@ -838,13 +835,13 @@
                     DataOutput dos = abvs.getDataOutput();
                     try {
                         AInt32 ai = new AInt32(i);
-                        AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(ai.getType()).serialize(
-                                ai, dos);
+                        AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(ai.getType()).serialize(ai,
+                                dos);
                     } catch (HyracksDataException e) {
                         throw new AlgebricksException(e);
                     }
-                    ICopyEvaluatorFactory fldIndexEvalFactory = new ConstantEvalFactory(Arrays.copyOf(
-                            abvs.getByteArray(), abvs.getLength()));
+                    ICopyEvaluatorFactory fldIndexEvalFactory = new ConstantEvalFactory(
+                            Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                     ICopyEvaluatorFactory evalFactory = new FieldAccessByIndexEvalFactory(recordEvalFactory,
                             fldIndexEvalFactory, recType);
                     IFunctionInfo finfoAccess = AsterixBuiltinFunctions
@@ -852,8 +849,8 @@
 
                     ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
                             new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                            new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                                    new AInt32(i)))));
+                            new MutableObject<ILogicalExpression>(
+                                    new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
                     return new Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory,
                             partitionFun, recType.getFieldTypes()[i]);
                 }
@@ -903,12 +900,15 @@
     }
 
     interface FunctionTypeInferer {
-        void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException;
+        void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                throws AlgebricksException;
     }
 
     void registerTypeInferers() {
         functionTypeInferers.put(AsterixBuiltinFunctions.LISTIFY, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
                 if (f.getArguments().size() == 0) {
                     ((ListifyAggregateDescriptor) fd).reset(new AOrderedListType(null, null));
@@ -926,7 +926,9 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.RECORD_MERGE, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
                 IAType outType = (IAType) context.getType(expr);
                 IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
@@ -935,7 +937,9 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.CAST_RECORD, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
                 ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType(funcExpr);
                 IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
@@ -946,7 +950,9 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.CAST_LIST, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
                 AbstractCollectionType rt = (AbstractCollectionType) TypeComputerUtilities.getRequiredType(funcExpr);
                 IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
@@ -957,13 +963,18 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.FLOW_RECORD, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
-                ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
+                ARecordType it = (ARecordType) TypeComputerUtilities
+                        .getInputType((AbstractFunctionCallExpression) expr);
                 ((FlowRecordDescriptor) fd).reset(it);
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 ARecordType rt = (ARecordType) context.getType(expr);
                 ((OpenRecordConstructorDescriptor) fd).reset(rt,
                         computeOpenFields((AbstractFunctionCallExpression) expr, rt));
@@ -976,8 +987,8 @@
                     Mutable<ILogicalExpression> argRef = expr.getArguments().get(2 * i);
                     ILogicalExpression arg = argRef.getValue();
                     if (arg.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                        String fn = ((AString) ((AsterixConstantValue) ((ConstantExpression) arg).getValue()).getObject())
-                                .getStringValue();
+                        String fn = ((AString) ((AsterixConstantValue) ((ConstantExpression) arg).getValue())
+                                .getObject()).getStringValue();
                         open[i] = true;
                         for (String s : recType.getFieldNames()) {
                             if (s.equals(fn)) {
@@ -993,22 +1004,30 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 ((ClosedRecordConstructorDescriptor) fd).reset((ARecordType) context.getType(expr));
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 ((OrderedListConstructorDescriptor) fd).reset((AOrderedListType) context.getType(expr));
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 ((UnorderedListConstructorDescriptor) fd).reset((AUnorderedListType) context.getType(expr));
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
                 IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
                 switch (t.getTypeTag()) {
@@ -1036,11 +1055,13 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.FIELD_ACCESS_NESTED, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
                 IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
-                AOrderedList fieldPath = (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce.getArguments()
-                        .get(1).getValue()).getValue()).getObject());
+                AOrderedList fieldPath = (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) fce
+                        .getArguments().get(1).getValue()).getValue()).getObject());
                 List<String> listFieldPath = new ArrayList<String>();
                 for (int i = 0; i < fieldPath.size(); i++) {
                     listFieldPath.add(((AString) fieldPath.getItem(i)).getStringValue());
@@ -1059,7 +1080,9 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.GET_RECORD_FIELDS, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
                 IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
                 if (t.getTypeTag().equals(ATypeTag.RECORD)) {
@@ -1071,7 +1094,9 @@
             }
         });
         functionTypeInferers.put(AsterixBuiltinFunctions.GET_RECORD_FIELD_VALUE, new FunctionTypeInferer() {
-            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context) throws AlgebricksException {
+            @Override
+            public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
+                    throws AlgebricksException {
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
                 IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
                 if (t.getTypeTag().equals(ATypeTag.RECORD)) {
@@ -1134,35 +1159,6 @@
     }
 
     @Override
-    public ITupleParserFactory createTupleParser(ARecordType recType, IParseFileSplitsDecl decl) {
-        return createTupleParser(recType, decl.isDelimitedFileFormat(), decl.getDelimChar(), decl.getQuote(),
-                decl.getHasHeader());
-    }
-
-    @Override
-    public ITupleParserFactory createTupleParser(ARecordType recType, boolean delimitedFormat, char delimiter,
-            char quote, boolean hasHeader) {
-        Map<String, String> conf = new HashMap<String, String>();
-        AsterixTupleParserFactory.InputDataFormat inputFormat = null;
-        if (delimitedFormat) {
-            conf.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT);
-            conf.put(AsterixTupleParserFactory.KEY_DELIMITER, "" + delimiter);
-            inputFormat = InputDataFormat.DELIMITED;
-        } else {
-            conf.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_ADM);
-            inputFormat = InputDataFormat.ADM;
-        }
-
-        if (hasHeader) {
-            conf.put(AsterixTupleParserFactory.HAS_HEADER,
-                    hasHeader ? Boolean.TRUE.toString() : Boolean.FALSE.toString());
-        }
-        conf.put(AsterixTupleParserFactory.KEY_QUOTE, "" + quote);
-        return new AsterixTupleParserFactory(conf, recType, inputFormat);
-    }
-
-
-    @Override
     public INullWriterFactory getNullWriterFactory() {
         return AqlNullWriterFactory.INSTANCE;
     }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractTupleParser.java
deleted file mode 100644
index f3199e9..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractTupleParser.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-
-/**
- * An abstract class implementation for ITupleParser. It provides common
- * functionality involved in parsing data in an external format and packing
- * frames with formed tuples.
- */
-public abstract class AbstractTupleParser implements ITupleParser {
-
-    protected static Logger LOGGER = Logger.getLogger(AbstractTupleParser.class.getName());
-
-    protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
-    protected DataOutput dos = tb.getDataOutput();
-    protected final ARecordType recType;
-    protected final IHyracksCommonContext ctx;
-
-    public AbstractTupleParser(IHyracksCommonContext ctx, ARecordType recType) throws HyracksDataException {
-        this.recType = recType;
-        this.ctx = ctx;
-    }
-
-    public abstract IDataParser getDataParser();
-
-    public abstract ITupleForwardPolicy getTupleParserPolicy();
-
-    @Override
-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-        IDataParser parser = getDataParser();
-        ITupleForwardPolicy policy = getTupleParserPolicy();
-        try {
-            parser.initialize(in, recType, true);
-            policy.initialize(ctx, writer);
-            while (true) {
-                tb.reset();
-                if (!parser.parse(tb.getDataOutput())) {
-                    break;
-                }
-                tb.addFieldEndOffset();
-                policy.addTuple(tb);
-            }
-            policy.close();
-        } catch (AsterixException ae) {
-            throw new HyracksDataException(ae);
-        } catch (IOException ioe) {
-            throw new HyracksDataException(ioe);
-        }
-    }
-
-}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AsterixTupleParserFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AsterixTupleParserFactory.java
deleted file mode 100644
index 2053a75..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AsterixTupleParserFactory.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.FeedPolicyAccessor;
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.asterix.common.parse.ITupleForwardPolicy.TupleForwardPolicyType;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.dataflow.std.file.ITupleParser;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public class AsterixTupleParserFactory implements ITupleParserFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public static enum InputDataFormat {
-        ADM,
-        DELIMITED,
-        UNKNOWN
-    }
-
-    public static final String HAS_HEADER = "has.header";
-    public static final String KEY_FORMAT = "format";
-    public static final String FORMAT_ADM = "adm";
-    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
-    public static final String FORMAT_BINARY = "binary";
-
-    public static final String KEY_PATH = "path";
-    public static final String KEY_SOURCE_DATATYPE = "type-name";
-    public static final String KEY_DELIMITER = "delimiter";
-    public static final String KEY_PARSER_FACTORY = "parser";
-    public static final String KEY_HEADER = "header";
-    public static final String KEY_QUOTE = "quote";
-    public static final String TIME_TRACKING = "time.tracking";
-    public static final String DEFAULT_QUOTE = "\"";
-    public static final String AT_LEAST_ONE_SEMANTICS = FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS;
-    public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
-    public static final String DEFAULT_DELIMITER = ",";
-
-    private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
-
-    private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
-        Map<ATypeTag, IValueParserFactory> m = new HashMap<ATypeTag, IValueParserFactory>();
-        m.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
-        m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        m.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
-        m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-        return m;
-    }
-
-    private final ARecordType recordType;
-    private final Map<String, String> configuration;
-    private final InputDataFormat inputDataFormat;
-
-    public AsterixTupleParserFactory(Map<String, String> configuration, ARecordType recType, InputDataFormat dataFormat) {
-        this.recordType = recType;
-        this.configuration = configuration;
-        this.inputDataFormat = dataFormat;
-    }
-
-    @Override
-    public ITupleParser createTupleParser(IHyracksCommonContext ctx) throws HyracksDataException {
-        ITupleParser tupleParser = null;
-        try {
-            String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
-            ITupleParserFactory parserFactory = null;
-            if (parserFactoryClassname != null) {
-                parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
-                tupleParser = parserFactory.createTupleParser(ctx);
-            } else {
-                IDataParser dataParser = null;
-                dataParser = createDataParser(ctx);
-                ITupleForwardPolicy policy = getTupleParserPolicy(configuration);
-                policy.configure(configuration);
-                tupleParser = new GenericTupleParser(ctx, recordType, dataParser, policy);
-            }
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-        return tupleParser;
-    }
-
-    private static class GenericTupleParser extends AbstractTupleParser {
-
-        private final IDataParser dataParser;
-
-        private final ITupleForwardPolicy policy;
-
-        public GenericTupleParser(IHyracksCommonContext ctx, ARecordType recType, IDataParser dataParser,
-                ITupleForwardPolicy policy) throws HyracksDataException {
-            super(ctx, recType);
-            this.dataParser = dataParser;
-            this.policy = policy;
-        }
-
-        @Override
-        public IDataParser getDataParser() {
-            return dataParser;
-        }
-
-        @Override
-        public ITupleForwardPolicy getTupleParserPolicy() {
-            return policy;
-        }
-
-    }
-
-    private IDataParser createDataParser(IHyracksCommonContext ctx) throws Exception {
-        IDataParser dataParser = null;
-        switch (inputDataFormat) {
-            case ADM:
-                dataParser = new ADMDataParser();
-                break;
-            case DELIMITED:
-                dataParser = configureDelimitedDataParser(ctx);
-                break;
-            case UNKNOWN:
-                String specifiedFormat = (String) configuration.get(KEY_FORMAT);
-                if (specifiedFormat == null) {
-                    throw new IllegalArgumentException(" Unspecified data format");
-                } else {
-                    if (FORMAT_ADM.equalsIgnoreCase(specifiedFormat.toUpperCase())) {
-                        dataParser = new ADMDataParser();
-                    } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat.toUpperCase())) {
-                        dataParser = configureDelimitedDataParser(ctx);
-                    } else {
-                        throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT)
-                                + " not supported");
-                    }
-                }
-        }
-        return dataParser;
-    }
-
-    public static ITupleForwardPolicy getTupleParserPolicy(Map<String, String> configuration) {
-        ITupleForwardPolicy policy = null;
-        ITupleForwardPolicy.TupleForwardPolicyType policyType = null;
-        String propValue = configuration.get(ITupleForwardPolicy.PARSER_POLICY);
-        if (propValue == null) {
-            policyType = TupleForwardPolicyType.FRAME_FULL;
-        } else {
-            policyType = TupleForwardPolicyType.valueOf(propValue.trim().toUpperCase());
-        }
-        switch (policyType) {
-            case FRAME_FULL:
-                policy = new FrameFullTupleForwardPolicy();
-                break;
-            case COUNTER_TIMER_EXPIRED:
-                policy = new CounterTimerTupleForwardPolicy();
-                break;
-            case RATE_CONTROLLED:
-                policy = new RateControlledTupleForwardPolicy();
-                break;
-        }
-        return policy;
-    }
-
-    private IDataParser configureDelimitedDataParser(IHyracksCommonContext ctx) throws AsterixException {
-        IValueParserFactory[] valueParserFactories = getValueParserFactories();
-        Character delimiter = getDelimiter(configuration);
-        char quote = getQuote(configuration, delimiter);
-        boolean hasHeader = hasHeader();
-        return new DelimitedDataParser(recordType, valueParserFactories, delimiter, quote, hasHeader);
-    }
-  
-
-    private boolean hasHeader() {
-        String value = configuration.get(KEY_HEADER);
-        if (value != null) {
-            return Boolean.valueOf(value);
-        }
-        return false;
-    }
-
-    private IValueParserFactory[] getValueParserFactories() {
-        int n = recordType.getFieldTypes().length;
-        IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
-        for (int i = 0; i < n; i++) {
-            ATypeTag tag = null;
-            if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
-                List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
-                if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
-                    throw new NotImplementedException("Non-optional UNION type is not supported.");
-                }
-                tag = unionTypes.get(1).getTypeTag();
-            } else {
-                tag = recordType.getFieldTypes()[i].getTypeTag();
-            }
-            if (tag == null) {
-                throw new NotImplementedException("Failed to get the type information for field " + i + ".");
-            }
-            IValueParserFactory vpf = valueParserFactoryMap.get(tag);
-            if (vpf == null) {
-                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-            }
-            fieldParserFactories[i] = vpf;
-        }
-        return fieldParserFactories;
-    }
-
-    // Get a delimiter from the given configuration
-    public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
-        String delimiterValue = configuration.get(AsterixTupleParserFactory.KEY_DELIMITER);
-        if (delimiterValue == null) {
-            delimiterValue = AsterixTupleParserFactory.DEFAULT_DELIMITER;
-        } else if (delimiterValue.length() != 1) {
-            throw new AsterixException("'" + delimiterValue
-                    + "' is not a valid delimiter. The length of a delimiter should be 1.");
-        }
-        return delimiterValue.charAt(0);
-    }
-
-    // Get a quote from the given configuration when the delimiter is given
-    // Need to pass delimiter to check whether they share the same character
-    public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
-        String quoteValue = configuration.get(AsterixTupleParserFactory.KEY_QUOTE);
-        if (quoteValue == null) {
-            quoteValue = AsterixTupleParserFactory.DEFAULT_QUOTE;
-        } else if (quoteValue.length() != 1) {
-            throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
-        }
-
-        // Since delimiter (char type value) can't be null,
-        // we only check whether delimiter and quote use the same character
-        if (quoteValue.charAt(0) == delimiter) {
-            throw new AsterixException("Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter
-                    + "'. ");
-        }
-
-        return quoteValue.charAt(0);
-    }
-
-    // Get the header flag
-    public static boolean getHasHeader(Map<String, String> configuration) {
-        return Boolean.parseBoolean(configuration.get(AsterixTupleParserFactory.KEY_HEADER));
-    }
-
-}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/DelimitedDataParser.java
deleted file mode 100644
index ca7c5c4..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/DelimitedDataParser.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.ANullSerializerDeserializer;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.std.file.FieldCursorForDelimitedDataParser;
-
-public class DelimitedDataParser extends AbstractDataParser implements IDataParser {
-
-    protected final IValueParserFactory[] valueParserFactories;
-    protected final char fieldDelimiter;
-    protected final char quote;
-    protected final boolean hasHeader;
-    protected final ARecordType recordType;
-    private IARecordBuilder recBuilder;
-    private ArrayBackedValueStorage fieldValueBuffer;
-    private DataOutput fieldValueBufferOutput;
-    private IValueParser[] valueParsers;
-    private FieldCursorForDelimitedDataParser cursor;
-    private byte[] fieldTypeTags;
-    private int[] fldIds;
-    private ArrayBackedValueStorage[] nameBuffers;
-    private boolean areAllNullFields;
-
-    public DelimitedDataParser(ARecordType recordType, IValueParserFactory[] valueParserFactories, char fieldDelimter,
-            char quote, boolean hasHeader) {
-        this.recordType = recordType;
-        this.valueParserFactories = valueParserFactories;
-        this.fieldDelimiter = fieldDelimter;
-        this.quote = quote;
-        this.hasHeader = hasHeader;
-    }
-
-    @Override
-    public void initialize(InputStream in, ARecordType recordType, boolean datasetRec) throws AsterixException,
-            IOException {
-
-        valueParsers = new IValueParser[valueParserFactories.length];
-        for (int i = 0; i < valueParserFactories.length; ++i) {
-            valueParsers[i] = valueParserFactories[i].createValueParser();
-        }
-
-        fieldValueBuffer = new ArrayBackedValueStorage();
-        fieldValueBufferOutput = fieldValueBuffer.getDataOutput();
-        recBuilder = new RecordBuilder();
-        recBuilder.reset(recordType);
-        recBuilder.init();
-
-        int n = recordType.getFieldNames().length;
-        fieldTypeTags = new byte[n];
-        for (int i = 0; i < n; i++) {
-            ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
-            fieldTypeTags[i] = tag.serialize();
-        }
-
-        fldIds = new int[n];
-        nameBuffers = new ArrayBackedValueStorage[n];
-        AMutableString str = new AMutableString(null);
-        for (int i = 0; i < n; i++) {
-            String name = recordType.getFieldNames()[i];
-            fldIds[i] = recBuilder.getFieldId(name);
-            if (fldIds[i] < 0) {
-                if (!recordType.isOpen()) {
-                    throw new HyracksDataException("Illegal field " + name + " in closed type " + recordType);
-                } else {
-                    nameBuffers[i] = new ArrayBackedValueStorage();
-                    fieldNameToBytes(name, str, nameBuffers[i]);
-                }
-            }
-        }
-
-        cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote);
-    }
-
-    @Override
-    public boolean parse(DataOutput out) throws AsterixException, IOException {
-        if (hasHeader && cursor.recordCount == 0) {
-            // Consume all fields of first record
-            cursor.nextRecord();
-            while (cursor.nextField());
-        }
-        while (cursor.nextRecord()) {
-            recBuilder.reset(recordType);
-            recBuilder.init();
-            areAllNullFields = true;
-
-            for (int i = 0; i < valueParsers.length; ++i) {
-                if (!cursor.nextField()) {
-                    break;
-                }
-                fieldValueBuffer.reset();
-
-                if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
-                        && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
-                    // if the field is empty and the type is optional, insert
-                    // NULL. Note that string type can also process empty field as an
-                    // empty string
-                    if (!NonTaggedFormatUtil.isOptional(recordType.getFieldTypes()[i])) {
-                        throw new AsterixException("At record: " + cursor.recordCount + " - Field " + cursor.fieldCount
-                                + " is not an optional type so it cannot accept null value. ");
-                    }
-                    fieldValueBufferOutput.writeByte(ATypeTag.NULL.serialize());
-                    ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
-                } else {
-                    fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
-                    // Eliminate doule quotes in the field that we are going to parse
-                    if (cursor.isDoubleQuoteIncludedInThisField) {
-                        cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
-                        cursor.fEnd -= cursor.doubleQuoteCount;
-                        cursor.isDoubleQuoteIncludedInThisField = false;
-                    }
-                    valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart,
-                            fieldValueBufferOutput);
-                    areAllNullFields = false;
-                }
-                if (fldIds[i] < 0) {
-                    recBuilder.addField(nameBuffers[i], fieldValueBuffer);
-                } else {
-                    recBuilder.addField(fldIds[i], fieldValueBuffer);
-                }
-            }
-
-            if (!areAllNullFields) {
-                recBuilder.write(out, true);
-                return true;
-            }
-        }
-        return false;
-    }
-
-    protected void fieldNameToBytes(String fieldName, AMutableString str, ArrayBackedValueStorage buffer)
-            throws HyracksDataException {
-        buffer.reset();
-        DataOutput out = buffer.getDataOutput();
-        str.setValue(fieldName);
-        try {
-            stringSerde.serialize(str, out);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java
deleted file mode 100644
index e22180c..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.util.Map;
-
-import org.apache.asterix.common.parse.ITupleForwardPolicy;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-
-public class FrameFullTupleForwardPolicy implements ITupleForwardPolicy {
-
-	private FrameTupleAppender appender;
-	private IFrame frame;
-	private IFrameWriter writer;
-
-	public void configure(Map<String, String> configuration) {
-		// no-op
-	}
-
-	public void initialize(IHyracksCommonContext ctx, IFrameWriter writer)
-			throws HyracksDataException {
-		this.appender = new FrameTupleAppender();
-		this.frame = new VSizeFrame(ctx);
-		this.writer = writer;
-		appender.reset(frame, true);
-	}
-
-	public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
-		boolean success = appender.append(tb.getFieldEndOffsets(),
-				tb.getByteArray(), 0, tb.getSize());
-		if (!success) {
-			FrameUtils.flushFrame(frame.getBuffer(), writer);
-			appender.reset(frame, true);
-			success = appender.append(tb.getFieldEndOffsets(),
-					tb.getByteArray(), 0, tb.getSize());
-			if (!success) {
-				throw new IllegalStateException();
-			}
-		}
-	}
-
-	public void close() throws HyracksDataException {
-		if (appender.getTupleCount() > 0) {
-			FrameUtils.flushFrame(frame.getBuffer(), writer);
-		}
-
-	}
-
-	@Override
-	public TupleForwardPolicyType getType() {
-		return TupleForwardPolicyType.FRAME_FULL;
-	}
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/IDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/IDataParser.java
deleted file mode 100644
index ba90e6c..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/IDataParser.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.types.ARecordType;
-
-/**
- * Interface implemented by a parser
- */
-public interface IDataParser {
-
-    /**
-     * Initialize the parser prior to actual parsing.
-     * 
-     * @param in
-     *            input stream to be parsed
-     * @param recordType
-     *            record type associated with input data
-     * @param datasetRec
-     *            boolean flag set to true if input data represents dataset
-     *            records.
-     * @throws AsterixException
-     * @throws IOException
-     */
-    public void initialize(InputStream in, ARecordType recordType, boolean datasetRec) throws AsterixException,
-            IOException;
-
-    /**
-     * Parse data from source input stream and output ADM records.
-     * 
-     * @param out
-     *            DataOutput instance that for writing the parser output.
-     * @return
-     * @throws AsterixException
-     * @throws IOException
-     */
-    public boolean parse(DataOutput out) throws AsterixException, IOException;
-}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateContolledParserPolicy.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateContolledParserPolicy.java
deleted file mode 100644
index 7b5d331..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/RateContolledParserPolicy.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.operators.file;
-
-import java.util.Map;
-
-import org.apache.asterix.common.parse.ITupleParserPolicy;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-
-public class RateContolledParserPolicy implements ITupleParserPolicy {
-
-    protected FrameTupleAppender appender;
-    protected IFrame  frame;
-    private IFrameWriter writer;
-    private long interTupleInterval;
-    private boolean delayConfigured;
-
-    public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
-
-    public RateContolledParserPolicy() {
-
-    }
-
-    public TupleParserPolicy getType() {
-        return ITupleParserPolicy.TupleParserPolicy.FRAME_FULL;
-    }
-
- 
-    @Override
-    public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
-        if (delayConfigured) {
-            try {
-                Thread.sleep(interTupleInterval);
-            } catch (InterruptedException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        boolean success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-        if (!success) {
-            FrameUtils.flushFrame(frame.getBuffer(), writer);
-            appender.reset(frame, true);
-            success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-            if (!success) {
-                throw new IllegalStateException();
-            }
-        }
-        appender.reset(frame, true);
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (appender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(frame.getBuffer(), writer);
-        }
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws HyracksDataException {
-        String propValue = configuration.get(INTER_TUPLE_INTERVAL);
-        if (propValue != null) {
-            interTupleInterval = Long.parseLong(propValue);
-        } else {
-            interTupleInterval = 0;
-        }
-        delayConfigured = interTupleInterval != 0;
-        
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx, IFrameWriter writer) throws HyracksDataException {
-        this.appender = new FrameTupleAppender();
-        this.frame = new VSizeFrame(ctx);
-    }
-}
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GULongIDGenerator.java b/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GULongIDGenerator.java
deleted file mode 100644
index 00f3bcb..0000000
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/GULongIDGenerator.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.tools.external.data;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class GULongIDGenerator {
-
-    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 long getNextULong() {
-        return nextValue.incrementAndGet();
-    }
-
-    public int getPartition() {
-        return partition;
-    }
-
-}
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
deleted file mode 100644
index 679f1af..0000000
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.tools.external.data;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Map;
-
-import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * An adapter that simulates a feed from the contents of a source file. The file
- * can be on the local file system or on HDFS. The feed ends when the content of
- * the source file has been ingested.
- */
-
-public class RateControlledFileSystemBasedAdapter extends FileSystemBasedAdapter implements IFeedAdapter {
-
-    private static final long serialVersionUID = 1L;
-    private FileSystemBasedAdapter coreAdapter;
-
-    public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, String> configuration,
-            FileSystemBasedAdapter coreAdapter, String format, ITupleParserFactory parserFactory,
-            IHyracksTaskContext ctx) throws Exception {
-        super(parserFactory, atype, ctx);
-        this.coreAdapter = coreAdapter;
-    }
-
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
-        return coreAdapter.getInputStream(partition);
-    }
-
-    @Override
-    public void stop() {
-       // ((RateControlledTupleParser) tupleParser).stop();
-    }
-
-    @Override
-    public DataExchangeMode getDataExchangeMode() {
-        return DataExchangeMode.PULL;
-    }
-
-    @Override
-    public boolean handleException(Exception e) {
-        return false;
-    }
-
-}
diff --git a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
deleted file mode 100644
index a8c77ac..0000000
--- a/asterix-tools/src/main/java/org/apache/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.tools.external.data;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
-import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.external.adapter.factory.HDFSAdapterFactory;
-import org.apache.asterix.external.adapter.factory.IAdapterFactory;
-import org.apache.asterix.external.adapter.factory.IFeedAdapterFactory;
-import org.apache.asterix.external.adapter.factory.NCFileSystemAdapterFactory;
-import org.apache.asterix.external.adapter.factory.StreamBasedAdapterFactory;
-import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.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 RateControlledFileSystemBasedAdapterFactory extends StreamBasedAdapterFactory implements
-        IFeedAdapterFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final String KEY_FILE_SYSTEM = "fs";
-    public static final String LOCAL_FS = "localfs";
-    public static final String HDFS = "hdfs";
-    public static final String KEY_PATH = "path";
-    public static final String KEY_FORMAT = "format";
-
-    private IAdapterFactory adapterFactory;
-    private String format;
-    private ARecordType atype;
-
-    @Override
-    public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
-        FileSystemBasedAdapter coreAdapter = (FileSystemBasedAdapter) adapterFactory.createAdapter(ctx, partition);
-        return new RateControlledFileSystemBasedAdapter(atype, configuration, coreAdapter, format, parserFactory, ctx);
-    }
-
-    @Override
-    public String getName() {
-        return "file_feed";
-    }
-
-    private void checkRequiredArgs(Map<String, String> configuration) throws Exception {
-        if (configuration.get(KEY_FILE_SYSTEM) == null) {
-            throw new Exception("File system type not specified. (fs=?) File system could be 'localfs' or 'hdfs'");
-        }
-        if (configuration.get(IAdapterFactory.KEY_TYPE_NAME) == null) {
-            throw new Exception("Record type not specified (type-name=?)");
-        }
-        if (configuration.get(KEY_PATH) == null) {
-            throw new Exception("File path not specified (path=?)");
-        }
-        if (configuration.get(KEY_FORMAT) == null) {
-            throw new Exception("File format not specified (format=?)");
-        }
-    }
-
-    @Override
-    public SupportedOperation getSupportedOperations() {
-        return SupportedOperation.READ;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
-        this.configuration = configuration;
-        checkRequiredArgs(configuration);
-        String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
-        String adapterFactoryClass = null;
-        if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
-            adapterFactoryClass = NCFileSystemAdapterFactory.class.getName();
-        } else if (fileSystem.equals(HDFS)) {
-            adapterFactoryClass = HDFSAdapterFactory.class.getName();
-        } else {
-            throw new AsterixException("Unsupported file system type " + fileSystem);
-        }
-        this.atype = outputType;
-        format = configuration.get(KEY_FORMAT);
-        adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
-        adapterFactory.configure(configuration, outputType);
-        configureFormat(outputType);
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return adapterFactory.getPartitionConstraint();
-    }
-
-    @Override
-    public ARecordType getAdapterOutputType() {
-        return atype;
-    }
-
-    @Override
-    public InputDataFormat getInputDataFormat() {
-        return InputDataFormat.UNKNOWN;
-    }
-
-    public boolean isRecordTrackingEnabled() {
-        return false;
-    }
-
-    public IIntakeProgressTracker createIntakeProgressTracker() {
-        throw new UnsupportedOperationException("Tracking of ingested records not enabled");
-    }
-
-}