You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2023/03/24 22:42:28 UTC

[asterixdb] branch master updated: [NO ISSUE][OTH] Remove external indexing runtime

This is an automated email from the ASF dual-hosted git repository.

mhubail pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new cb74ca85f8 [NO ISSUE][OTH] Remove external indexing runtime
cb74ca85f8 is described below

commit cb74ca85f8f7dbe67c7cce63cb07d521acc8abb5
Author: Murtadha Hubail <mu...@couchbase.com>
AuthorDate: Fri Mar 24 19:51:24 2023 +0300

    [NO ISSUE][OTH] Remove external indexing runtime
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Change-Id: I6fb6cd6070329ac966cbe89c8e0ea9966f702947
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17445
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../asterix/app/io/PersistedResourceRegistry.java  |   6 -
 .../asterix/app/translator/QueryTranslator.java    | 384 +-----------
 .../hyracks/bootstrap/GlobalRecoveryManager.java   | 100 ---
 .../TestLsmBTreeResourceFactoryProvider.java       |   7 +-
 .../leftouterjoin-rtree.1.ddl.sqlpp                |  63 --
 .../leftouterjoin-rtree.2.update.sqlpp             |  19 -
 .../leftouterjoin-rtree.3.query.sqlpp              |  39 --
 .../leftouterjoin/leftouterjoin.1.ddl.sqlpp        |  61 --
 .../leftouterjoin/leftouterjoin.2.update.sqlpp     |  25 -
 .../leftouterjoin/leftouterjoin.3.query.sqlpp      |  32 -
 .../rtree-index/rtree-index.1.ddl.sqlpp            |  42 --
 .../rtree-index/rtree-index.2.update.sqlpp         |  19 -
 .../rtree-index/rtree-index.3.query.sqlpp          |  27 -
 .../sequence-format/sequence-format.1.ddl.sqlpp    |  43 --
 .../sequence-format/sequence-format.2.update.sqlpp |  25 -
 .../sequence-format/sequence-format.3.query.sqlpp  |  33 -
 .../text-format/text-format.1.ddl.sqlpp            |  43 --
 .../text-format/text-format.2.update.sqlpp         |  26 -
 .../text-format/text-format.3.query.sqlpp          |  34 -
 .../adm-format/adm-format.1.ddl.sqlpp              |  48 --
 .../adm-format/adm-format.2.update.sqlpp           |  25 -
 .../adm-format/adm-format.3.query.sqlpp            |  32 -
 .../leftouterjoin-rtree.1.ddl.sqlpp                |  59 --
 .../leftouterjoin-rtree.2.update.sqlpp             |  19 -
 .../leftouterjoin-rtree.3.query.sqlpp              |  39 --
 .../leftouterjoin/leftouterjoin.1.ddl.sqlpp        |  58 --
 .../leftouterjoin/leftouterjoin.2.update.sqlpp     |  25 -
 .../leftouterjoin/leftouterjoin.3.query.sqlpp      |  32 -
 .../rtree-index/rtree-index.1.ddl.sqlpp            |  41 --
 .../rtree-index/rtree-index.2.update.sqlpp         |  19 -
 .../rtree-index/rtree-index.3.query.sqlpp          |  27 -
 .../decorrelate_with_unique_id_2.1.ddl.sqlpp       |   6 -
 .../leftouterjoin-rtree/leftouterjoin-rtree.1.adm  |   9 -
 .../leftouterjoin/leftouterjoin.1.adm              |   9 -
 .../external-indexing/rc-format/rc-format.1.adm    |   2 -
 .../rtree-index/rtree-index.1.adm                  |   2 -
 .../sequence-format/sequence-format.1.adm          |   2 -
 .../text-format/text-format.1.adm                  |   2 -
 .../resources/runtimets/testsuite_sqlpp_hdfs.xml   |  99 ---
 .../resources/runtimets/testsuite_sqlpp_parser.xml |  98 ---
 .../adapter/factory/LookupAdapterFactory.java      |  91 ---
 .../asterix/external/api/ILookupReaderFactory.java |  27 -
 .../asterix/external/api/ILookupRecordReader.java  |  40 --
 .../external/dataset/adapter/LookupAdapter.java    | 171 -----
 .../indexing/ExternalFileIndexAccessor.java        | 148 -----
 .../indexing/FileIndexTupleTranslator.java         |  86 ---
 .../external/indexing/FileOffsetIndexer.java       |  79 ---
 .../external/indexing/FilesIndexDescription.java   |  92 ---
 .../external/indexing/IndexingScheduler.java       | 324 ----------
 .../external/indexing/RCRecordIdReader.java        |  43 --
 .../external/indexing/RecordColumnarIndexer.java   |  94 ---
 .../apache/asterix/external/indexing/RecordId.java |  54 --
 .../asterix/external/indexing/RecordIdReader.java  |  78 ---
 .../external/indexing/RecordIdReaderFactory.java   |  38 --
 .../external/input/HDFSDataSourceFactory.java      |  12 +-
 .../hdfs/AbstractCharRecordLookupReader.java       |  79 ---
 .../hdfs/AbstractHDFSLookupRecordReader.java       | 113 ----
 .../reader/hdfs/HDFSLookupReaderFactory.java       | 103 ---
 .../record/reader/hdfs/SequenceLookupReader.java   |  72 ---
 .../input/record/reader/hdfs/TextLookupReader.java |  65 --
 ...ctExternalDatasetIndexesOperatorDescriptor.java |  96 ---
 .../ExternalBTreeSearchOperatorDescriptor.java     |  56 --
 .../ExternalBTreeSearchOperatorNodePushable.java   |  94 ---
 ...ernalDatasetIndexesAbortOperatorDescriptor.java |  51 --
 ...rnalDatasetIndexesCommitOperatorDescriptor.java |  57 --
 ...nalDatasetIndexesRecoverOperatorDescriptor.java |  50 --
 ...ExternalFilesIndexCreateOperatorDescriptor.java | 119 ----
 ...alFilesIndexModificationOperatorDescriptor.java | 129 ----
 .../ExternalIndexBulkLoadOperatorDescriptor.java   |  55 --
 .../ExternalIndexBulkLoadOperatorNodePushable.java |  61 --
 .../ExternalIndexBulkModifyOperatorDescriptor.java |  53 --
 ...xternalIndexBulkModifyOperatorNodePushable.java | 118 ----
 .../ExternalLookupOperatorDescriptor.java          | 133 ----
 .../ExternalRTreeSearchOperatorDescriptor.java     |  57 --
 .../ExternalRTreeSearchOperatorNodePushable.java   |  95 ---
 .../external/provider/AdapterFactoryProvider.java  |  31 -
 .../external/provider/ExternalIndexerProvider.java |  41 --
 .../provider/LookupReaderFactoryProvider.java      |  46 --
 .../apache/asterix/external/util/HDFSUtils.java    |  27 -
 .../declared/BTreeResourceFactoryProvider.java     |  23 +-
 .../metadata/declared/MetadataProvider.java        |  47 +-
 .../apache/asterix/metadata/entities/Dataset.java  |  50 +-
 .../metadata/utils/ExternalIndexingOperations.java | 551 ----------------
 .../utils/RTreeResourceFactoryProvider.java        |  21 +-
 .../utils/SecondaryBTreeOperationsHelper.java      |  53 --
 .../utils/SecondaryIndexOperationsHelper.java      |  78 ---
 .../utils/SecondaryRTreeOperationsHelper.java      |  60 --
 .../external_index/external_index.1.script.sqlpp   |  19 -
 .../external_index/external_index.2.ddl.sqlpp      |  46 --
 .../external_index/external_index.3.script.sqlpp   |  19 -
 .../external_index/external_index.4.query.sqlpp    |  31 -
 .../external_index/external_index.5.script.sqlpp   |  19 -
 .../src/test/resources/transactionts/testsuite.xml |   5 -
 .../btree/dataflow/ExternalBTreeLocalResource.java | 104 ---
 .../ExternalBTreeLocalResourceFactory.java         |  67 --
 .../ExternalBTreeWithBuddyLocalResource.java       | 107 ----
 ...ExternalBTreeWithBuddyLocalResourceFactory.java |  67 --
 .../storage/am/lsm/btree/impls/ExternalBTree.java  | 604 ------------------
 .../am/lsm/btree/impls/ExternalBTreeOpContext.java |  82 ---
 .../am/lsm/btree/impls/ExternalBTreeWithBuddy.java | 698 ---------------------
 .../impls/ExternalBTreeWithBuddyOpContext.java     |  84 ---
 .../storage/am/lsm/btree/impls/LSMBTree.java       |  19 -
 .../storage/am/lsm/btree/utils/LSMBTreeUtil.java   | 115 ----
 .../am/lsm/common/impls/AbstractLSMIndex.java      |  36 --
 .../am/lsm/common/impls/ExternalIndexHarness.java  | 332 ----------
 .../rtree/dataflow/ExternalRTreeLocalResource.java | 114 ----
 .../ExternalRTreeLocalResourceFactory.java         |  72 ---
 .../am/lsm/rtree/impls/AbstractLSMRTree.java       |  28 -
 .../storage/am/lsm/rtree/impls/ExternalRTree.java  | 656 -------------------
 .../am/lsm/rtree/impls/ExternalRTreeOpContext.java |  77 ---
 .../storage/am/lsm/rtree/impls/LSMRTree.java       |  22 -
 .../storage/am/lsm/rtree/utils/LSMRTreeUtils.java  |  60 --
 112 files changed, 30 insertions(+), 9195 deletions(-)

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 82c70a3ef7..077b657e33 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -105,8 +105,6 @@ import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.storage.am.common.data.PointablePrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.column.dataflow.LSMColumnBTreeLocalResource;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResource;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResource;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
 import org.apache.hyracks.storage.am.lsm.common.impls.ConcurrentMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
@@ -123,7 +121,6 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8Word
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeLocalResource;
 import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeLocalResource;
 import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterLocalResource;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -161,9 +158,6 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
         registeredClasses.put("LSMRTreeLocalResource", LSMRTreeLocalResource.class);
         registeredClasses.put("LSMRTreeWithAntiMatterLocalResource", LSMRTreeWithAntiMatterLocalResource.class);
         registeredClasses.put("LSMInvertedIndexLocalResource", LSMInvertedIndexLocalResource.class);
-        registeredClasses.put("ExternalBTreeLocalResource", ExternalBTreeLocalResource.class);
-        registeredClasses.put("ExternalBTreeWithBuddyLocalResource", ExternalBTreeWithBuddyLocalResource.class);
-        registeredClasses.put("ExternalRTreeLocalResource", ExternalRTreeLocalResource.class);
         registeredClasses.put("LSMColumnBTreeLocalResource", LSMColumnBTreeLocalResource.class);
 
         // ILSMMergePolicyFactory
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 775672cbe2..399b7b4e1a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -33,7 +33,6 @@ import java.util.Date;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -71,7 +70,6 @@ import org.apache.asterix.common.api.IRequestTracker;
 import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -96,7 +94,6 @@ import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
-import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -148,7 +145,6 @@ import org.apache.asterix.lang.common.statement.LoadStatement;
 import org.apache.asterix.lang.common.statement.NodeGroupDropStatement;
 import org.apache.asterix.lang.common.statement.NodegroupDecl;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.RefreshExternalDatasetStatement;
 import org.apache.asterix.lang.common.statement.SetStatement;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.statement.StopFeedStatement;
@@ -193,7 +189,6 @@ import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
-import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -263,7 +258,6 @@ import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.util.LogRedactionUtil;
-import org.apache.hyracks.util.OptionalBoolean;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -1558,76 +1552,16 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         Index filesIndex = null;
         boolean firstExternalDatasetIndex = false;
         boolean datasetLocked = false;
-        List<ExternalFile> externalFilesSnapshot;
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
         JobSpecification spec;
-        boolean filesIndexReplicated = false;
         try {
             index.setPendingOp(MetadataUtil.PENDING_ADD_OP);
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 validateDatasetState(metadataProvider, ds, sourceLoc);
-            } else {
+            } else if (ds.getDatasetType() == DatasetType.EXTERNAL) {
                 // External dataset
-                // Check if the dataset is indexible
-                if (!ExternalIndexingOperations.isIndexible((ExternalDatasetDetails) ds.getDatasetDetails())) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            dataset() + " using " + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter()
-                                    + " adapter can't be indexed");
-                }
-                // Check if the name of the index is valid
-                if (!ExternalIndexingOperations.isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            "external " + dataset() + " index name is invalid");
-                }
-
-                // Check if the files index exist
-                filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
-                        index.getDataverseName(), index.getDatasetName(),
-                        IndexingConstants.getFilesIndexName(index.getDatasetName()));
-                firstExternalDatasetIndex = filesIndex == null;
-                // Lock external dataset
-                ExternalDatasetsRegistry.INSTANCE.buildIndexBegin(ds, firstExternalDatasetIndex);
-                datasetLocked = true;
-                if (firstExternalDatasetIndex) {
-                    // Verify that no one has created an index before we acquire the lock
-                    filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
-                            index.getDataverseName(), index.getDatasetName(),
-                            IndexingConstants.getFilesIndexName(index.getDatasetName()));
-                    if (filesIndex != null) {
-                        ExternalDatasetsRegistry.INSTANCE.buildIndexEnd(ds, firstExternalDatasetIndex);
-                        firstExternalDatasetIndex = false;
-                        ExternalDatasetsRegistry.INSTANCE.buildIndexBegin(ds, firstExternalDatasetIndex);
-                    }
-                }
-                if (firstExternalDatasetIndex) {
-                    // Get snapshot from External File System
-                    externalFilesSnapshot = ExternalIndexingOperations.getSnapshotFromExternalFileSystem(ds);
-                    // Add an entry for the files index
-                    OptionalBoolean excludeUnknownKey =
-                            ((Index.ValueIndexDetails) index.getIndexDetails()).getExcludeUnknownKey();
-                    OptionalBoolean castDefaultNull =
-                            ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDefaultNull();
-                    String datetimeFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDatetimeFormat();
-                    String dateFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDateFormat();
-                    String timeFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastTimeFormat();
-
-                    filesIndex = new Index(index.getDataverseName(), index.getDatasetName(),
-                            IndexingConstants.getFilesIndexName(index.getDatasetName()), IndexType.BTREE,
-                            new Index.ValueIndexDetails(ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
-                                    ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, excludeUnknownKey,
-                                    castDefaultNull, datetimeFormat, dateFormat, timeFormat),
-                            false, false, MetadataUtil.PENDING_ADD_OP);
-                    MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
-                    // Add files to the external files index
-                    for (ExternalFile file : externalFilesSnapshot) {
-                        MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-                    }
-                    // This is the first index for the external dataset, replicate the files index
-                    spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot,
-                            metadataProvider);
-                    filesIndexReplicated = true;
-                    runJob(hcc, spec, jobFlags);
-                }
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, dataset() + " using "
+                        + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter() + " adapter can't be indexed");
             }
 
             // check whether there exists another enforced index on the same field
@@ -1748,24 +1682,6 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            // If files index was replicated for external dataset, it should be cleaned up
-            // on NC side
-            if (filesIndexReplicated) {
-                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                bActiveTxn = true;
-                try {
-                    JobSpecification jobSpec =
-                            ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds);
-                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                    bActiveTxn = false;
-                    runJob(hcc, jobSpec, jobFlags);
-                } catch (Exception e2) {
-                    e.addSuppressed(e2);
-                    if (bActiveTxn) {
-                        abort(e, e2, mdTxnCtx);
-                    }
-                }
-            }
 
             if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
                 // #. execute compensation operations
@@ -1992,13 +1908,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     case EXTERNAL:
                         indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                         for (Index index : indexes) {
-                            if (ExternalIndexingOperations.isFileIndex(index)) {
-                                jobsToExecute.add(ExternalIndexingOperations
-                                        .buildDropFilesIndexJobSpec(metadataProvider, dataset));
-                            } else {
-                                jobsToExecute.add(
-                                        IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
-                            }
+                            jobsToExecute
+                                    .add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
                         }
                         externalDatasetsToDeregister.add(dataset);
                         break;
@@ -2299,67 +2210,6 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
             } else {
                 // External dataset
-                indexName = stmtIndexDrop.getIndexName().getValue();
-                Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
-                if (index == null) {
-                    if (stmtIndexDrop.getIfExists()) {
-                        MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                        return false;
-                    } else {
-                        throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName);
-                    }
-                } else if (ExternalIndexingOperations.isFileIndex(index)) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            "Dropping " + dataset() + " files index is not allowed.");
-                }
-                ensureNonPrimaryIndexDrop(index, sourceLoc);
-                prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName, jobsToExecute,
-                        mdTxnCtx, ds, index);
-
-                List<Index> datasetIndexes =
-                        MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
-                if (datasetIndexes.size() == 2) {
-                    dropFilesIndex = true;
-                    // only one index + the files index, we need to delete both of the indexes
-                    for (Index externalIndex : datasetIndexes) {
-                        if (ExternalIndexingOperations.isFileIndex(externalIndex)) {
-                            jobsToExecute
-                                    .add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds));
-                            // #. mark PendingDropOp on the existing files index
-                            MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
-                                    externalIndex.getIndexName());
-                            MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                                    new Index(dataverseName, datasetName, externalIndex.getIndexName(),
-                                            externalIndex.getIndexType(), externalIndex.getIndexDetails(),
-                                            externalIndex.isEnforced(), externalIndex.isPrimaryIndex(),
-                                            MetadataUtil.PENDING_DROP_OP));
-                        }
-                    }
-                }
-
-                // #. commit the existing transaction before calling runJob.
-                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                bActiveTxn = false;
-                progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
-
-                for (JobSpecification jobSpec : jobsToExecute) {
-                    runJob(hcc, jobSpec);
-                }
-
-                // #. begin a new transaction
-                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                bActiveTxn = true;
-                metadataProvider.setMetadataTxnContext(mdTxnCtx);
-
-                // #. finally, delete the existing index
-                MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
-                if (dropFilesIndex) {
-                    // delete the files index too
-                    MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
-                            IndexingConstants.getFilesIndexName(datasetName));
-                    MetadataManager.INSTANCE.dropDatasetExternalFiles(mdTxnCtx, ds);
-                    ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
-                }
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return true;
@@ -4897,229 +4747,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
 
     protected void handleExternalDatasetRefreshStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
-        RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
-        SourceLocation sourceLoc = stmtRefresh.getSourceLocation();
-        DataverseName dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
-        String datasetName = stmtRefresh.getDatasetName().getValue();
-        TransactionState transactionState = TransactionState.COMMIT;
-        JobSpecification spec = null;
-        Dataset ds = null;
-        List<ExternalFile> metadataFiles = null;
-        List<ExternalFile> deletedFiles = null;
-        List<ExternalFile> addedFiles = null;
-        List<ExternalFile> appendedFiles = null;
-        List<Index> indexes = null;
-        Dataset transactionDataset = null;
-        boolean lockAquired = false;
-        boolean success = false;
-        if (isCompileOnly()) {
-            return;
-        }
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        boolean bActiveTxn = true;
-        lockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
-        try {
-            ds = metadataProvider.findDataset(dataverseName, datasetName);
-            // Dataset exists ?
-            if (ds == null) {
-                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
-                        dataverseName);
-            }
-            // Dataset external ?
-            if (ds.getDatasetType() != DatasetType.EXTERNAL) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, dataset() + " " + datasetName
-                        + " in " + dataverse() + " " + dataverseName + " is not an external " + dataset());
-            }
-            // Dataset has indexes ?
-            indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
-            if (indexes.isEmpty()) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "External " + dataset() + " "
-                        + datasetName + " in " + dataverse() + " " + dataverseName + " doesn't have any index");
-            }
-
-            // Record transaction time
-            Date txnTime = new Date();
-
-            // refresh lock here
-            ExternalDatasetsRegistry.INSTANCE.refreshBegin(ds);
-            lockAquired = true;
-
-            // Get internal files
-            metadataFiles = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, ds);
-            deletedFiles = new ArrayList<>();
-            addedFiles = new ArrayList<>();
-            appendedFiles = new ArrayList<>();
-
-            // Compute delta
-            // Now we compare snapshot with external file system
-            if (ExternalIndexingOperations.isDatasetUptodate(ds, metadataFiles, addedFiles, deletedFiles,
-                    appendedFiles)) {
-                ((ExternalDatasetDetails) ds.getDatasetDetails()).setRefreshTimestamp(txnTime);
-                MetadataManager.INSTANCE.updateDataset(mdTxnCtx, ds);
-                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                // latch will be released in the finally clause
-                return;
-            }
-
-            // At this point, we know data has changed in the external file system, record
-            // transaction in metadata and start
-            transactionDataset = ExternalIndexingOperations.createTransactionDataset(ds);
-            /*
-             * Remove old dataset record and replace it with a new one
-             */
-            MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
-
-            // Add delta files to the metadata
-            for (ExternalFile file : addedFiles) {
-                MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-            }
-            for (ExternalFile file : appendedFiles) {
-                MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-            }
-            for (ExternalFile file : deletedFiles) {
-                MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-            }
-
-            // Create the files index update job
-            spec = ExternalIndexingOperations.buildFilesIndexUpdateOp(ds, metadataFiles, addedFiles, appendedFiles,
-                    metadataProvider);
-
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            bActiveTxn = false;
-            transactionState = TransactionState.BEGIN;
-
-            // run the files update job
-            runJob(hcc, spec);
-
-            for (Index index : indexes) {
-                if (!ExternalIndexingOperations.isFileIndex(index)) {
-                    spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, addedFiles,
-                            appendedFiles, metadataProvider, sourceLoc);
-                    // run the files update job
-                    runJob(hcc, spec);
-                }
-            }
-
-            // all index updates has completed successfully, record transaction state
-            spec = ExternalIndexingOperations.buildCommitJob(ds, indexes, metadataProvider);
-
-            // Aquire write latch again -> start a transaction and record the decision to
-            // commit
-            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-            metadataProvider.setMetadataTxnContext(mdTxnCtx);
-            bActiveTxn = true;
-            ((ExternalDatasetDetails) transactionDataset.getDatasetDetails())
-                    .setState(TransactionState.READY_TO_COMMIT);
-            ((ExternalDatasetDetails) transactionDataset.getDatasetDetails()).setRefreshTimestamp(txnTime);
-            MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            bActiveTxn = false;
-            transactionState = TransactionState.READY_TO_COMMIT;
-            // We don't release the latch since this job is expected to be quick
-            runJob(hcc, spec);
-            // Start a new metadata transaction to record the final state of the transaction
-            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-            metadataProvider.setMetadataTxnContext(mdTxnCtx);
-            bActiveTxn = true;
-
-            for (ExternalFile file : metadataFiles) {
-                if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
-                    MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                } else if (file.getPendingOp() == ExternalFilePendingOp.NO_OP) {
-                    Iterator<ExternalFile> iterator = appendedFiles.iterator();
-                    while (iterator.hasNext()) {
-                        ExternalFile appendedFile = iterator.next();
-                        if (file.getFileName().equals(appendedFile.getFileName())) {
-                            // delete existing file
-                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                            // delete existing appended file
-                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, appendedFile);
-                            // add the original file with appended information
-                            appendedFile.setFileNumber(file.getFileNumber());
-                            appendedFile.setPendingOp(ExternalFilePendingOp.NO_OP);
-                            MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, appendedFile);
-                            iterator.remove();
-                        }
-                    }
-                }
-            }
-
-            // remove the deleted files delta
-            for (ExternalFile file : deletedFiles) {
-                MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-            }
-
-            // insert new files
-            for (ExternalFile file : addedFiles) {
-                MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                file.setPendingOp(ExternalFilePendingOp.NO_OP);
-                MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-            }
-
-            // mark the transaction as complete
-            ((ExternalDatasetDetails) transactionDataset.getDatasetDetails()).setState(TransactionState.COMMIT);
-            MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
-
-            // commit metadata transaction
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            success = true;
-        } catch (Exception e) {
-            if (bActiveTxn) {
-                abort(e, e, mdTxnCtx);
-            }
-            if (transactionState == TransactionState.READY_TO_COMMIT) {
-                throw new IllegalStateException("System is inconsistent state: commit of (" + dataverseName + "."
-                        + datasetName + ") refresh couldn't carry out the commit phase", e);
-            }
-            if (transactionState == TransactionState.COMMIT) {
-                // Nothing to do , everything should be clean
-                throw e;
-            }
-            if (transactionState == TransactionState.BEGIN) {
-                // transaction failed, need to do the following
-                // clean NCs removing transaction components
-                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                bActiveTxn = true;
-                metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                spec = ExternalIndexingOperations.buildAbortOp(ds, indexes, metadataProvider);
-                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                bActiveTxn = false;
-                try {
-                    runJob(hcc, spec);
-                } catch (Exception e2) {
-                    // This should never happen -- fix throw illegal
-                    e.addSuppressed(e2);
-                    throw new IllegalStateException("System is in inconsistent state. Failed to abort refresh", e);
-                }
-                // remove the delta of files
-                // return the state of the dataset to committed
-                try {
-                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                    for (ExternalFile file : deletedFiles) {
-                        MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                    }
-                    for (ExternalFile file : addedFiles) {
-                        MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                    }
-                    for (ExternalFile file : appendedFiles) {
-                        MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                    }
-                    MetadataManager.INSTANCE.updateDataset(mdTxnCtx, ds);
-                    // commit metadata transaction
-                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                } catch (Exception e2) {
-                    abort(e, e2, mdTxnCtx);
-                    e.addSuppressed(e2);
-                    throw new IllegalStateException("System is in inconsistent state. Failed to drop delta files", e);
-                }
-            }
-        } finally {
-            if (lockAquired) {
-                ExternalDatasetsRegistry.INSTANCE.refreshEnd(ds, success);
-            }
-            metadataProvider.getLocks().unlock();
-        }
+        // no op
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index e6ef8df9df..2452b8f421 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -30,23 +30,14 @@ import org.apache.asterix.app.message.StorageCleanupRequestMessage;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
-import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
-import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.utils.DatasetUtil;
-import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
-import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
@@ -162,7 +153,6 @@ public class GlobalRecoveryManager implements IGlobalRecoveryManager {
             throws Exception {
         // Loop over datasets
         for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
-            mdTxnCtx = recoverDatasets(appCtx, mdTxnCtx, dataverse);
             // Fixes ASTERIXDB-2386 by caching the dataverse during recovery
             MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse.getDataverseName());
         }
@@ -182,96 +172,6 @@ public class GlobalRecoveryManager implements IGlobalRecoveryManager {
         }
     }
 
-    private MetadataTransactionContext recoverDatasets(ICcApplicationContext appCtx,
-            MetadataTransactionContext mdTxnCtx, Dataverse dataverse) throws Exception {
-        if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
-            MetadataProvider metadataProvider = MetadataProvider.create(appCtx, dataverse);
-            try {
-                List<Dataset> datasets =
-                        MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverse.getDataverseName());
-                for (Dataset dataset : datasets) {
-                    if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-                        // External dataset
-                        // Get indexes
-                        List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
-                                dataset.getDataverseName(), dataset.getDatasetName());
-                        // Get the state of the dataset
-                        ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
-                        TransactionState datasetState = dsd.getState();
-                        if (!indexes.isEmpty()) {
-                            if (datasetState == TransactionState.BEGIN) {
-                                List<ExternalFile> files =
-                                        MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
-                                // if persumed abort, roll backward
-                                // 1. delete all pending files
-                                for (ExternalFile file : files) {
-                                    if (file.getPendingOp() != ExternalFilePendingOp.NO_OP) {
-                                        MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                                    }
-                                }
-                            }
-                            // 2. clean artifacts in NCs
-                            metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                            JobSpecification jobSpec =
-                                    ExternalIndexingOperations.buildAbortOp(dataset, indexes, metadataProvider);
-                            executeHyracksJob(jobSpec);
-                            // 3. correct the dataset state
-                            ((ExternalDatasetDetails) dataset.getDatasetDetails()).setState(TransactionState.COMMIT);
-                            MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
-                            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                        } else if (datasetState == TransactionState.READY_TO_COMMIT) {
-                            List<ExternalFile> files =
-                                    MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
-                            // if ready to commit, roll forward
-                            // 1. commit indexes in NCs
-                            metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                            JobSpecification jobSpec =
-                                    ExternalIndexingOperations.buildRecoverOp(dataset, indexes, metadataProvider);
-                            executeHyracksJob(jobSpec);
-                            // 2. add pending files in metadata
-                            for (ExternalFile file : files) {
-                                if (file.getPendingOp() == ExternalFilePendingOp.ADD_OP) {
-                                    MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                                    file.setPendingOp(ExternalFilePendingOp.NO_OP);
-                                    MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-                                } else if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
-                                    // find original file
-                                    for (ExternalFile originalFile : files) {
-                                        if (originalFile.getFileName().equals(file.getFileName())) {
-                                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, originalFile);
-                                            break;
-                                        }
-                                    }
-                                } else if (file.getPendingOp() == ExternalFilePendingOp.APPEND_OP) {
-                                    // find original file
-                                    for (ExternalFile originalFile : files) {
-                                        if (originalFile.getFileName().equals(file.getFileName())) {
-                                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                                            MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, originalFile);
-                                            originalFile.setSize(file.getSize());
-                                            MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, originalFile);
-                                        }
-                                    }
-                                }
-                                // 3. correct the dataset state
-                                ((ExternalDatasetDetails) dataset.getDatasetDetails())
-                                        .setState(TransactionState.COMMIT);
-                                MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
-                                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                            }
-                        }
-                    }
-                }
-            } finally {
-                metadataProvider.getLocks().unlock();
-            }
-        }
-        return mdTxnCtx;
-    }
-
     @Override
     public boolean isRecoveryCompleted() {
         return recoveryCompleted;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index fcc8bc6974..13b9bd1571 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -25,7 +25,6 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -96,7 +95,7 @@ public class TestLsmBTreeResourceFactoryProvider implements IResourceFactoryProv
             return primaryTypeTraits;
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-            return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
+            return null;
         }
         Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
@@ -131,7 +130,7 @@ public class TestLsmBTreeResourceFactoryProvider implements IResourceFactoryProv
             return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-            return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
+            return null;
         }
         Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
@@ -166,7 +165,7 @@ public class TestLsmBTreeResourceFactoryProvider implements IResourceFactoryProv
             return dataset.getPrimaryBloomFilterFields();
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-                return FilesIndexDescription.BLOOM_FILTER_FIELDS;
+                return null;
             } else {
                 return new int[] { ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size() };
             }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp
deleted file mode 100644
index c801bd04ed..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp
+++ /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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.TwitterUserType as
- closed {
-  `screen-name` : string,
-  lang : string,
-  `friends-count` : bigint,
-  `statuses-count` : bigint,
-  name : string,
-  `followers-count` : bigint
-};
-
-create type test.TweetMessageType as
- closed {
-  tweetid : bigint,
-  user : TwitterUserType,
-  `sender-location` : point,
-  `send-time` : datetime,
-  `referred-topics` : {{string}},
-  `message-text` : string,
-  countA : bigint,
-  countB : bigint
-};
-
-create external  dataset TweetMessages(TweetMessageType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/tw_for_indexleftouterjoin.adm`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index IdIx  on TweetMessages (tweetid) type btree;
-
-create  index msgCountAIx  on TweetMessages (countA) type btree;
-
-create  index msgCountBIx  on TweetMessages (countB) type btree;
-
-create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp
deleted file mode 100644
index bd244d07ab..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp
+++ /dev/null
@@ -1,19 +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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp
deleted file mode 100644
index 56615cee1b..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp
+++ /dev/null
@@ -1,39 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-use test;
-
-
-select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
-        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
-        from  TweetMessages as t2
-        where test.`spatial-intersect`(t2.`sender-location`,n)
-        order by t2.tweetid
-    )}
-from  TweetMessages as t1
-with  n as test.`create-circle`(t1.`sender-location`,0.5)
-where (t1.tweetid < test.bigint('10'))
-order by t1.tweetid
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp
deleted file mode 100644
index 6cbeb79f49..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp
+++ /dev/null
@@ -1,61 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.TwitterUserType as
- closed {
-  `screen-name` : string,
-  lang : string,
-  `friends-count` : bigint,
-  `statuses-count` : bigint,
-  name : string,
-  `followers-count` : bigint
-};
-
-create type test.TweetMessageType as
- closed {
-  tweetid : bigint,
-  user : TwitterUserType,
-  `sender-location` : point,
-  `send-time` : datetime,
-  `referred-topics` : {{string}},
-  `message-text` : string,
-  countA : bigint,
-  countB : bigint
-};
-
-create external  dataset TweetMessages(TweetMessageType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/tw_for_indexleftouterjoin.adm`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index IdIx  on TweetMessages (tweetid) type btree;
-
-create  index msgCountAIx  on TweetMessages (countA) type btree;
-
-create  index msgCountBIx  on TweetMessages (countB) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp
deleted file mode 100644
index df67ea335d..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp
+++ /dev/null
@@ -1,25 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp
deleted file mode 100644
index 86fc87bed3..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp
+++ /dev/null
@@ -1,32 +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.
- */
-
-use test;
-
-
-select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
-        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
-        from  TweetMessages as t2
-        where (t1.countA /*+ indexnl */  = t2.countB)
-        order by t2.tweetid
-    )}
-from  TweetMessages as t1
-where (t1.tweetid < test.bigint('10'))
-order by t1.tweetid
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp
deleted file mode 100644
index 6e87e383f1..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,42 +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.
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create external  dataset MyData(MyRecord) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/spatialData.json`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index rtree_index_point  on MyData (point) type rtree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.2.update.sqlpp
deleted file mode 100644
index bd244d07ab..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.2.update.sqlpp
+++ /dev/null
@@ -1,19 +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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.3.query.sqlpp
deleted file mode 100644
index 6406409d2a..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rtree-index/rtree-index.3.query.sqlpp
+++ /dev/null
@@ -1,27 +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.
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.1.ddl.sqlpp
deleted file mode 100644
index f366b4a6c0..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.1.ddl.sqlpp
+++ /dev/null
@@ -1,43 +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.
- */
-/*
- * Description  : Create an external dataset that contains records stored with sequence hdfs file format.
- *                Build an index over the external dataset age attribute
- *                Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date         : 3rd Jan 2014
-*/
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.EmployeeType as
- closed {
-  id : bigint,
-  name : string,
-  age : bigint
-};
-
-create external  dataset EmployeeDataset(EmployeeType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/external-indexing-test.seq`),(`input-format`=`sequence-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
-
-create  index EmployeeAgeIdx  on EmployeeDataset (age) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.2.update.sqlpp
deleted file mode 100644
index bbb644e285..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.2.update.sqlpp
+++ /dev/null
@@ -1,25 +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.
- */
-/*
- * Description  : Create an external dataset that contains records stored with sequence hdfs file format.
- *                Build an index over the external dataset age attribute
- *                Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date         : 3rd Jan 2014
-*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.3.query.sqlpp
deleted file mode 100644
index 477ad06a70..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/sequence-format/sequence-format.3.query.sqlpp
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Description  : Create an external dataset that contains records stored with sequence hdfs file format.
- *                Build an index over the external dataset age attribute
- *                Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date         : 3rd Jan 2014
-*/
-
-use test;
-
-
-select element emp
-from  EmployeeDataset as emp
-where (emp.age = 22)
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.1.ddl.sqlpp
deleted file mode 100644
index ca874e54bf..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.1.ddl.sqlpp
+++ /dev/null
@@ -1,43 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.EmployeeType as
- closed {
-  id : bigint,
-  name : string,
-  age : bigint
-};
-
-create external  dataset EmployeeDataset(EmployeeType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/external-indexing-test.txt`),(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
-
-create  index EmployeeAgeIdx  on EmployeeDataset (age) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.2.update.sqlpp
deleted file mode 100644
index 1c90e5364a..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.2.update.sqlpp
+++ /dev/null
@@ -1,26 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.3.query.sqlpp
deleted file mode 100644
index b0ba55d064..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/text-format/text-format.3.query.sqlpp
+++ /dev/null
@@ -1,34 +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.
- */
-/*
- * Description  : Create an external dataset that contains records stored with text hdfs file format.
- *                Build an index over the external dataset age attribute
- *                Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date         : 3rd Jan 2014
-*/
-
-use test;
-
-
-select element emp
-from  EmployeeDataset as emp
-where (emp.age = 22)
-order by emp.id
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.1.ddl.sqlpp
deleted file mode 100644
index 7cafed2abf..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.1.ddl.sqlpp
+++ /dev/null
@@ -1,48 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
-{
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create external  dataset MyData(MyRecord) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/spatialData.json`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index idx  on MyData (id:bigint?) type btree enforced;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.2.update.sqlpp
deleted file mode 100644
index 7b2e6a3d1c..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.2.update.sqlpp
+++ /dev/null
@@ -1,25 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.3.query.sqlpp
deleted file mode 100644
index dfa9fe32d4..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/adm-format/adm-format.3.query.sqlpp
+++ /dev/null
@@ -1,32 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-use test;
-
-
-select element d
-from  MyData as d
-where (d.id = 10)
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp
deleted file mode 100644
index f7000eb1d1..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.sqlpp
+++ /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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.TwitterUserType as
- closed {
-  `screen-name` : string,
-  lang : string,
-  `friends-count` : bigint,
-  `statuses-count` : bigint,
-  name : string,
-  `followers-count` : bigint
-};
-
-create type test.TweetMessageType as
-{
-  user : TwitterUserType,
-  `send-time` : datetime,
-  `referred-topics` : {{string}},
-  `message-text` : string
-};
-
-create external  dataset TweetMessages(TweetMessageType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/tw_for_indexleftouterjoin.adm`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index IdIx  on TweetMessages (tweetid:bigint?) type btree enforced;
-
-create  index msgCountAIx  on TweetMessages (countA:bigint?) type btree enforced;
-
-create  index msgCountBIx  on TweetMessages (countB:bigint?) type btree enforced;
-
-create  index twmSndLocIx  on TweetMessages (`sender-location`:point?) type rtree enforced;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp
deleted file mode 100644
index bd244d07ab..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.sqlpp
+++ /dev/null
@@ -1,19 +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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp
deleted file mode 100644
index 56615cee1b..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.sqlpp
+++ /dev/null
@@ -1,39 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-use test;
-
-
-select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
-        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
-        from  TweetMessages as t2
-        where test.`spatial-intersect`(t2.`sender-location`,n)
-        order by t2.tweetid
-    )}
-from  TweetMessages as t1
-with  n as test.`create-circle`(t1.`sender-location`,0.5)
-where (t1.tweetid < test.bigint('10'))
-order by t1.tweetid
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp
deleted file mode 100644
index fcd4f77686..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.1.ddl.sqlpp
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.TwitterUserType as
- closed {
-  `screen-name` : string,
-  lang : string,
-  `friends-count` : bigint,
-  `statuses-count` : bigint,
-  name : string,
-  `followers-count` : bigint
-};
-
-create type test.TweetMessageType as
-{
-  user : TwitterUserType,
-  `sender-location` : point,
-  `send-time` : datetime,
-  `referred-topics` : {{string}},
-  `message-text` : string
-};
-
-create external  dataset TweetMessages(TweetMessageType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/tw_for_indexleftouterjoin.adm`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index IdIx  on TweetMessages (tweetid:bigint?) type btree enforced;
-
-create  index msgCountAIx  on TweetMessages (countA:bigint?) type btree enforced;
-
-create  index msgCountBIx  on TweetMessages (countB:bigint?) type btree enforced;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp
deleted file mode 100644
index df67ea335d..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.2.update.sqlpp
+++ /dev/null
@@ -1,25 +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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp
deleted file mode 100644
index 86fc87bed3..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/leftouterjoin/leftouterjoin.3.query.sqlpp
+++ /dev/null
@@ -1,32 +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.
- */
-
-use test;
-
-
-select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
-        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
-        from  TweetMessages as t2
-        where (t1.countA /*+ indexnl */  = t2.countB)
-        order by t2.tweetid
-    )}
-from  TweetMessages as t1
-where (t1.tweetid < test.bigint('10'))
-order by t1.tweetid
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp
deleted file mode 100644
index 9b947590ad..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,41 +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.
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
-{
-  id : bigint,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create external  dataset MyData(MyRecord) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/spatialData.json`),(`input-format`=`text-input-format`),(`format`=`adm`));
-
-create  index rtree_index_point  on MyData (point:point?) type rtree enforced;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.2.update.sqlpp
deleted file mode 100644
index bd244d07ab..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.2.update.sqlpp
+++ /dev/null
@@ -1,19 +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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.3.query.sqlpp
deleted file mode 100644
index 6406409d2a..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/external-indexing/rtree-index/rtree-index.3.query.sqlpp
+++ /dev/null
@@ -1,27 +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.
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/decorrelate_with_unique_id_2/decorrelate_with_unique_id_2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/decorrelate_with_unique_id_2/decorrelate_with_unique_id_2.1.ddl.sqlpp
index d12411637e..aea6acc593 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/decorrelate_with_unique_id_2/decorrelate_with_unique_id_2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/decorrelate_with_unique_id_2/decorrelate_with_unique_id_2.1.ddl.sqlpp
@@ -47,9 +47,3 @@ create type test.TweetMessageType as
 
 create external  dataset TweetMessages(TweetMessageType) using `hdfs`((`hdfs`=`hdfs://127.0.0.1:31888`),(`path`=`/asterix/tw_for_indexleftouterjoin.adm`),(`input-format`=`text-input-format`),(`format`=`adm`));
 
-create  index IdIx  on TweetMessages (tweetid) type btree;
-
-create  index msgCountAIx  on TweetMessages (countA) type btree;
-
-create  index msgCountBIx  on TweetMessages (countB) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm
deleted file mode 100644
index d3eb71a784..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm
+++ /dev/null
@@ -1,9 +0,0 @@
-{ "tweetid1": 1, "loc1": point("42.83,72.44"), "nearby-message": [ { "tweetid2": 1, "loc2": point("42.83,72.44") }, { "tweetid2": 55, "loc2": point("42.77,72.16") }, { "tweetid2": 114, "loc2": point("42.87,72.38") } ] }
-{ "tweetid1": 2, "loc1": point("34.81,72.44"), "nearby-message": [ { "tweetid2": 2, "loc2": point("34.81,72.44") } ] }
-{ "tweetid1": 3, "loc1": point("24.54,82.66"), "nearby-message": [ { "tweetid2": 3, "loc2": point("24.54,82.66") } ] }
-{ "tweetid1": 4, "loc1": point("38.14,68.1"), "nearby-message": [ { "tweetid2": 4, "loc2": point("38.14,68.1") } ] }
-{ "tweetid1": 5, "loc1": point("35.4,68.89"), "nearby-message": [ { "tweetid2": 5, "loc2": point("35.4,68.89") } ] }
-{ "tweetid1": 6, "loc1": point("42.75,78.5"), "nearby-message": [ { "tweetid2": 6, "loc2": point("42.75,78.5") } ] }
-{ "tweetid1": 7, "loc1": point("48.16,71.59"), "nearby-message": [ { "tweetid2": 7, "loc2": point("48.16,71.59") }, { "tweetid2": 42, "loc2": point("47.86,71.93") }, { "tweetid2": 192, "loc2": point("48.12,72.0") } ] }
-{ "tweetid1": 8, "loc1": point("36.17,72.56"), "nearby-message": [ { "tweetid2": 8, "loc2": point("36.17,72.56") } ] }
-{ "tweetid1": 9, "loc1": point("38.02,70.38"), "nearby-message": [ { "tweetid2": 9, "loc2": point("38.02,70.38") }, { "tweetid2": 51, "loc2": point("37.65,70.54") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm
deleted file mode 100644
index 9a236f622b..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm
+++ /dev/null
@@ -1,9 +0,0 @@
-{ "tweetid1": 1, "count1": 1, "t2info": [  ] }
-{ "tweetid1": 2, "count1": 2, "t2info": [ { "tweetid2": 60, "count2": 2 } ] }
-{ "tweetid1": 3, "count1": 3, "t2info": [ { "tweetid2": 105, "count2": 3 }, { "tweetid2": 206, "count2": 3 } ] }
-{ "tweetid1": 4, "count1": 4, "t2info": [  ] }
-{ "tweetid1": 5, "count1": 5, "t2info": [ { "tweetid2": 138, "count2": 5 }, { "tweetid2": 175, "count2": 5 } ] }
-{ "tweetid1": 6, "count1": 6, "t2info": [ { "tweetid2": 148, "count2": 6 } ] }
-{ "tweetid1": 7, "count1": 7, "t2info": [ { "tweetid2": 125, "count2": 7 } ] }
-{ "tweetid1": 8, "count1": 8, "t2info": [  ] }
-{ "tweetid1": 9, "count1": 9, "t2info": [ { "tweetid2": 141, "count2": 9 } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm
deleted file mode 100644
index aca40dcee7..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "id": 3, "name": "Samuel", "age": 22 }
-{ "id": 10, "name": "David", "age": 22 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm
deleted file mode 100644
index b7c61805ab..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "id": 12 }
-{ "id": 20 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm
deleted file mode 100644
index aca40dcee7..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "id": 3, "name": "Samuel", "age": 22 }
-{ "id": 10, "name": "David", "age": 22 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm
deleted file mode 100644
index aca40dcee7..0000000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "id": 3, "name": "Samuel", "age": 22 }
-{ "id": 10, "name": "David", "age": 22 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
index 4d50ca2aa5..0d3ae3dddc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -17,78 +17,6 @@
  ! under the License.
  !-->
 <test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
-  <test-group name="open-index-enforced">
-    <test-group name="open-index-enforced/external-indexing">
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
-  </test-group>
-  <test-group name="nested-open-index">
-    <test-group name="nested-open-index/external-indexing">
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
-  </test-group>
-  <test-group name="nested-index">
-    <test-group name="nested-index/external-indexing">
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
-  </test-group>
   <test-group name="subquery">
     <test-case FilePath="subquery">
       <compilation-unit name="decorrelate_with_unique_id_2">
@@ -131,31 +59,4 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="external-indexing">
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="text-format">
-        <output-dir compare="Text">text-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="sequence-format">
-        <output-dir compare="Text">sequence-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rtree-index">
-        <output-dir compare="Text">rtree-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin">
-        <output-dir compare="Text">leftouterjoin</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin-rtree">
-        <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
index a96c57edb7..94ee14a7d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
@@ -3103,28 +3103,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="open-index-enforced/external-indexing">
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="AST">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="AST">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="AST">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="AST">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
   </test-group>
   <test-group name="nested-open-index">
     <test-group FilePath="nested-open-index/index-join">
@@ -3278,28 +3256,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="nested-open-index/external-indexing">
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="AST">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="AST">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="AST">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="AST">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
     <test-group name="nested-open-index/highly-open-highly-nested">
       <test-case FilePath="nested-open-index/highly-open-highly-nested">
         <compilation-unit name="bottom-closed-top-closed">
@@ -3530,28 +3486,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="nested-index/external-indexing">
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="AST">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="AST">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="AST">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="AST">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
   </test-group>
   <test-group name="nested-index-dml">
     <test-case FilePath="nested-index-dml">
@@ -6146,38 +6080,6 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="external-indexing">
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="text-format">
-        <output-dir compare="AST">text-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="sequence-format">
-        <output-dir compare="AST">sequence-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rc-format">
-        <output-dir compare="AST">rc-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rtree-index">
-        <output-dir compare="AST">rtree-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin">
-        <output-dir compare="AST">leftouterjoin</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin-rtree">
-        <output-dir compare="AST">leftouterjoin-rtree</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="temporal">
     <test-case FilePath="temporal">
       <compilation-unit name="overlap_bins_gby_3">
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
deleted file mode 100644
index fc0c4f9376..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
+++ /dev/null
@@ -1,91 +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.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.provider.LookupReaderFactoryProvider;
-import org.apache.asterix.external.provider.ParserFactoryProvider;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class LookupAdapterFactory<T> implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-    private IRecordDataParserFactory dataParserFactory;
-    private ILookupReaderFactory readerFactory;
-    private final ARecordType recordType;
-    private final int[] ridFields;
-    private Map<String, String> configuration;
-    private final boolean retainInput;
-    private final boolean retainMissing;
-    private final IMissingWriterFactory isMissingWriterFactory;
-
-    public LookupAdapterFactory(ARecordType recordType, int[] ridFields, boolean retainInput, boolean retainNull,
-            IMissingWriterFactory iNullWriterFactory) {
-        this.recordType = recordType;
-        this.ridFields = ridFields;
-        this.retainInput = retainInput;
-        this.retainMissing = retainNull;
-        this.isMissingWriterFactory = iNullWriterFactory;
-    }
-
-    public LookupAdapter<T> createAdapter(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecDesc,
-            ExternalFileIndexAccessor snapshotAccessor, IFrameWriter writer) throws HyracksDataException {
-        try {
-            IRecordDataParser<T> dataParser = dataParserFactory.createRecordParser(ctx);
-            ILookupRecordReader<? extends T> reader =
-                    readerFactory.createRecordReader(ctx, partition, snapshotAccessor);
-            reader.configure(configuration);
-            RecordIdReader ridReader = RecordIdReaderFactory.create(configuration, ridFields);
-            return new LookupAdapter<>(dataParser, reader, inRecDesc, ridReader, retainInput, retainMissing,
-                    isMissingWriterFactory, ctx, writer);
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    public void configure(ICCServiceContext serviceContext, Map<String, String> configuration,
-            IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
-        this.configuration = configuration;
-        readerFactory =
-                LookupReaderFactoryProvider.getLookupReaderFactory(serviceContext, configuration, warningCollector);
-        dataParserFactory = (IRecordDataParserFactory<T>) ParserFactoryProvider.getDataParserFactory(configuration);
-        dataParserFactory.setRecordType(recordType);
-        readerFactory.configure(serviceContext, configuration, warningCollector);
-        dataParserFactory.configure(configuration);
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java
deleted file mode 100644
index 28dbde764b..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupReaderFactory.java
+++ /dev/null
@@ -1,27 +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.api;
-
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public interface ILookupReaderFactory<T> extends IExternalDataSourceFactory {
-    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
-            ExternalFileIndexAccessor snapshotAccessor) throws Exception;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.java
deleted file mode 100644
index 2588f41f7e..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILookupRecordReader.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.api;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.external.indexing.RecordId;
-
-public interface ILookupRecordReader<T> {
-
-    public void configure(Map<String, String> configuration) throws Exception;
-
-    public Class<?> getRecordClass() throws IOException;
-
-    public IRawRecord<T> read(RecordId rid) throws Exception;
-
-    public void open() throws IOException;
-
-    public void fail() throws IOException;
-
-    public void close() throws IOException;
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
deleted file mode 100644
index 5700f9df27..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
+++ /dev/null
@@ -1,171 +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.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.IMissingWriter;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-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 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 missingTupleBuild;
-
-    public LookupAdapter(IRecordDataParser<T> dataParser, ILookupRecordReader<? extends T> recordReader,
-            RecordDescriptor inRecDesc, RecordIdReader ridReader, boolean propagateInput, boolean retainNull,
-            IMissingWriterFactory iNullWriterFactory, IHyracksTaskContext ctx, IFrameWriter writer)
-            throws HyracksDataException {
-        this.dataParser = dataParser;
-        this.recordReader = recordReader;
-        this.propagateInput = propagateInput;
-        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(IMissingWriterFactory iNullWriterFactory) {
-        if (propagateInput) {
-            // This LookupAdapter generates an external record as its output.
-            // Thus, we add 1.
-            tb = new ArrayTupleBuilder(tupleAccessor.getFieldCount() + 1);
-            frameTuple = new FrameTupleReference();
-        } else {
-            tb = new ArrayTupleBuilder(1);
-        }
-        if (retainNull) {
-            IMissingWriter missingWriter = iNullWriterFactory.createMissingWriter();
-            missingTupleBuild = new ArrayTupleBuilder(1);
-            DataOutput out = missingTupleBuild.getDataOutput();
-            try {
-                missingWriter.writeMissing(out);
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-        } else {
-            missingTupleBuild = null;
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        try {
-            recordReader.fail();
-        } catch (Throwable th) {
-            throw HyracksDataException.create(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) {
-                    propagateInputFields(tupleIndex);
-                }
-                if (record != null) {
-                    if (dataParser.parse(record, tb.getDataOutput())) {
-                        tb.addFieldEndOffset();
-                        DataflowUtils.addTupleToFrame(appender, tb, writer);
-                    }
-                } else if (retainNull) {
-                    tb.getDataOutput().write(missingTupleBuild.getByteArray());
-                    tb.addFieldEndOffset();
-                    DataflowUtils.addTupleToFrame(appender, tb, writer);
-                }
-                tupleIndex++;
-            }
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    private void propagateInputFields(int idx) throws IOException {
-        frameTuple.reset(tupleAccessor, idx);
-        for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-            tb.getDataOutput().write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
-                    frameTuple.getFieldLength(i));
-            tb.addFieldEndOffset();
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            appender.write(writer, true);
-        } catch (Exception e) {
-            writer.fail();
-            throw e;
-        } finally {
-            writer.close();
-        }
-    }
-
-    @Override
-    public void flush() throws HyracksDataException {
-        appender.flush(writer);
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
deleted file mode 100644
index 58a2413cf5..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
+++ /dev/null
@@ -1,148 +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;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.util.Date;
-
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.ARecord;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.util.CleanupUtils;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
-import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-/*
- * This class was created specifically to facilitate accessing
- * external file index when doing external lookup during runtime
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class ExternalFileIndexAccessor {
-    private IIndexDataflowHelper indexDataflowHelper;
-    private IHyracksTaskContext ctx;
-    private ExternalBTree index;
-    private ArrayTupleBuilder searchKeyTupleBuilder;
-    private ArrayTupleReference searchKey;
-    private AMutableInt32 currentFileNumber = new AMutableInt32(-1);
-    private ISerializerDeserializer intSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-    private RangePredicate searchPredicate;
-    private ILSMIndexAccessor fileIndexAccessor;
-    private IIndexCursor fileIndexSearchCursor;
-    private ISearchOperationCallbackFactory searchCallbackFactory;
-    private int version;
-    private ISerializerDeserializer externalFileRecordSerde = FilesIndexDescription.createExternalFileRecordSerde();
-
-    public ExternalFileIndexAccessor(IIndexDataflowHelper indexDataflowHelper,
-            ISearchOperationCallbackFactory searchCallbackFactory, int version) {
-        this.indexDataflowHelper = indexDataflowHelper;
-        this.searchCallbackFactory = searchCallbackFactory;
-        this.version = version;
-    }
-
-    public void open() throws HyracksDataException {
-        // Open the index and get the instance
-        indexDataflowHelper.open();
-        index = (ExternalBTree) indexDataflowHelper.getIndexInstance();
-        // 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());
-        MultiComparator 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 = searchCallbackFactory
-                .createSearchOperationCallback(indexDataflowHelper.getResource().getId(), ctx, null);
-        fileIndexAccessor = index.createAccessor(searchCallback, version);
-        fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
-    }
-
-    public void lookup(int fileId, ExternalFile file) throws HyracksDataException {
-        // Set search parameters
-        currentFileNumber.setValue(fileId);
-        searchKeyTupleBuilder.reset();
-        searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
-        searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
-        fileIndexSearchCursor.close();
-
-        // Perform search
-        fileIndexAccessor.search(fileIndexSearchCursor, searchPredicate);
-        if (fileIndexSearchCursor.hasNext()) {
-            fileIndexSearchCursor.next();
-            ITupleReference tuple = fileIndexSearchCursor.getTuple();
-            // Deserialize
-            byte[] serRecord = tuple.getFieldData(FilesIndexDescription.FILE_PAYLOAD_INDEX);
-            int recordStartOffset = tuple.getFieldStart(FilesIndexDescription.FILE_PAYLOAD_INDEX);
-            int recordLength = tuple.getFieldLength(FilesIndexDescription.FILE_PAYLOAD_INDEX);
-            ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
-            DataInput in = new DataInputStream(stream);
-            ARecord externalFileRecord = (ARecord) externalFileRecordSerde.deserialize(in);
-            setFile(externalFileRecord, file);
-        } else {
-            // This should never happen
-            throw new RuntimeDataException(ErrorCode.INDEXING_EXTERNAL_FILE_INDEX_ACCESSOR_UNABLE_TO_FIND_FILE_INDEX);
-        }
-    }
-
-    private void setFile(ARecord externalFileRecord, ExternalFile file) {
-        file.setFileName(
-                ((AString) externalFileRecord.getValueByPos(FilesIndexDescription.EXTERNAL_FILE_NAME_FIELD_INDEX))
-                        .getStringValue());
-        file.setSize(((AInt64) externalFileRecord.getValueByPos(FilesIndexDescription.EXTERNAL_FILE_SIZE_FIELD_INDEX))
-                .getLongValue());
-        file.setLastModefiedTime(new Date(
-                ((ADateTime) externalFileRecord.getValueByPos(FilesIndexDescription.EXTERNAL_FILE_MOD_DATE_FIELD_INDEX))
-                        .getChrononTime()));
-    }
-
-    public void close() throws HyracksDataException {
-        Throwable failure = ResourceReleaseUtils.close(fileIndexSearchCursor, null);
-        failure = CleanupUtils.destroy(failure, fileIndexSearchCursor, fileIndexAccessor);
-        failure = ResourceReleaseUtils.close(indexDataflowHelper, failure);
-        if (failure != null) {
-            throw HyracksDataException.create(failure);
-        }
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.java
deleted file mode 100644
index 2597782365..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileIndexTupleTranslator.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.external.indexing;
-
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-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.IAObject;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-@SuppressWarnings("unchecked")
-public class FileIndexTupleTranslator {
-    private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(FilesIndexDescription.FILE_INDEX_TUPLE_SIZE);
-    private RecordBuilder recordBuilder = new RecordBuilder();
-    private ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-    private AMutableInt32 aInt32 = new AMutableInt32(0);
-    private AMutableInt64 aInt64 = new AMutableInt64(0);
-    private AMutableString aString = new AMutableString(null);
-    private AMutableDateTime aDateTime = new AMutableDateTime(0);
-    private ISerializerDeserializer<IAObject> stringSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-    private ISerializerDeserializer<IAObject> dateTimeSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-    private ISerializerDeserializer<IAObject> longSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-    private ArrayTupleReference tuple = new ArrayTupleReference();
-
-    public ITupleReference getTupleFromFile(ExternalFile file) throws HyracksDataException {
-        tupleBuilder.reset();
-        //File Number
-        aInt32.setValue(file.getFileNumber());
-        FilesIndexDescription.FILE_NUMBER_SERDE.serialize(aInt32, tupleBuilder.getDataOutput());
-        tupleBuilder.addFieldEndOffset();
-
-        //File Record
-        recordBuilder.reset(FilesIndexDescription.EXTERNAL_FILE_RECORD_TYPE);
-        // write field 0 (File Name)
-        fieldValue.reset();
-        aString.setValue(file.getFileName());
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        recordBuilder.addField(0, fieldValue);
-
-        //write field 1 (File Size)
-        fieldValue.reset();
-        aInt64.setValue(file.getSize());
-        longSerde.serialize(aInt64, fieldValue.getDataOutput());
-        recordBuilder.addField(1, fieldValue);
-
-        //write field 2 (File Mod Date)
-        fieldValue.reset();
-        aDateTime.setValue(file.getLastModefiedTime().getTime());
-        dateTimeSerde.serialize(aDateTime, fieldValue.getDataOutput());
-        recordBuilder.addField(2, fieldValue);
-
-        //write the record
-        recordBuilder.write(tupleBuilder.getDataOutput(), true);
-        tupleBuilder.addFieldEndOffset();
-        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-        return tuple;
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
deleted file mode 100644
index 34c8795899..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
+++ /dev/null
@@ -1,79 +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;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.api.IExternalIndexer;
-import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-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.api.exceptions.HyracksDataException;
-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 final AMutableInt32 fileNumber = new AMutableInt32(0);
-    protected final AMutableInt64 offset = new AMutableInt64(0);
-    protected RecordReader<?, ? extends Writable> recordReader;
-
-    @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<IAObject> intSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-    @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<IAObject> longSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-
-    @Override
-    public void reset(IIndexingDatasource dataSource) throws HyracksDataException {
-        try {
-            //TODO: Make it more generic since we can't assume it is always going to be HDFS records
-            fileNumber.setValue(dataSource.getSnapshot().get(dataSource.getCurrentSplitIndex()).getFileNumber());
-            recordReader = dataSource.getReader();
-            offset.setValue(recordReader.getPos());
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public void index(ArrayTupleBuilder tb) throws HyracksDataException {
-        try {
-            tb.addField(intSerde, fileNumber);
-            tb.addField(longSerde, offset);
-            // Get position for next index(tb) call
-            offset.setValue(recordReader.getPos());
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public int getNumberOfFields() {
-        return NUM_OF_FIELDS;
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FilesIndexDescription.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FilesIndexDescription.java
deleted file mode 100644
index 45ccfa0f2c..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FilesIndexDescription.java
+++ /dev/null
@@ -1,92 +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;
-
-import java.io.IOException;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.formats.nontagged.TypeTraitProvider;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-
-@SuppressWarnings("rawtypes")
-public class FilesIndexDescription {
-    public static final int FILE_INDEX_TUPLE_SIZE = 2;
-    public static final int FILE_KEY_INDEX = 0;
-    public static final int FILE_KEY_SIZE = 1;
-    public static final int FILE_PAYLOAD_INDEX = 1;
-    private static final String[] payloadFieldNames = { "FileName", "FileSize", "FileModDate" };
-    private static final IAType[] payloadFieldTypes =
-            { BuiltinType.ASTRING, BuiltinType.AINT64, BuiltinType.ADATETIME };
-
-    public static final int[] BLOOM_FILTER_FIELDS = { 0 };
-    public static final int EXTERNAL_FILE_NAME_FIELD_INDEX = 0;
-    public static final int EXTERNAL_FILE_SIZE_FIELD_INDEX = 1;
-    public static final int EXTERNAL_FILE_MOD_DATE_FIELD_INDEX = 2;
-
-    public static final ARecordType EXTERNAL_FILE_RECORD_TYPE =
-            new ARecordType("ExternalFileRecordType", payloadFieldNames, payloadFieldTypes, true);
-    public static final ITypeTraits[] EXTERNAL_FILE_BUDDY_BTREE_TYPE_TRAITS =
-            new ITypeTraits[] { TypeTraitProvider.INSTANCE.getTypeTrait(IndexingConstants.FILE_NUMBER_FIELD_TYPE) };
-    public static final ITypeTraits[] EXTERNAL_FILE_INDEX_TYPE_TRAITS =
-            new ITypeTraits[] { TypeTraitProvider.INSTANCE.getTypeTrait(IndexingConstants.FILE_NUMBER_FIELD_TYPE),
-                    TypeTraitProvider.INSTANCE.getTypeTrait(EXTERNAL_FILE_RECORD_TYPE) };
-    public static final IBinaryComparatorFactory[] FILES_INDEX_COMP_FACTORIES = new IBinaryComparatorFactory[] {
-            BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true) };
-    public static final ISerializerDeserializer FILE_NUMBER_SERDE =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(IndexingConstants.FILE_NUMBER_FIELD_TYPE);
-    public static final ISerializerDeserializer[] EXTERNAL_FILE_BUDDY_BTREE_FIELDS =
-            new ISerializerDeserializer[] { FILE_NUMBER_SERDE };
-    public static final RecordDescriptor FILE_BUDDY_BTREE_RECORD_DESCRIPTOR =
-            new RecordDescriptor(EXTERNAL_FILE_BUDDY_BTREE_FIELDS, EXTERNAL_FILE_BUDDY_BTREE_TYPE_TRAITS);
-
-    private FilesIndexDescription() {
-    }
-
-    public static ISerializerDeserializer createExternalFileRecordSerde() {
-        return SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(EXTERNAL_FILE_RECORD_TYPE);
-    }
-
-    public static ISerializerDeserializer[] createExternalFileTupleFieldsSerdes() {
-        return new ISerializerDeserializer[] { FILE_NUMBER_SERDE, createExternalFileRecordSerde() };
-    }
-
-    public static RecordDescriptor createFileIndexRecordDescriptor() {
-        return new RecordDescriptor(createExternalFileTupleFieldsSerdes(), EXTERNAL_FILE_INDEX_TYPE_TRAITS);
-    }
-
-    @SuppressWarnings("unchecked")
-    public static void getBuddyBTreeTupleFromFileNumber(ArrayTupleReference tuple, ArrayTupleBuilder tupleBuilder,
-            AMutableInt32 aInt32) throws IOException, AsterixException {
-        tupleBuilder.reset();
-        FILE_NUMBER_SERDE.serialize(aInt32, tupleBuilder.getDataOutput());
-        tupleBuilder.addFieldEndOffset();
-        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
deleted file mode 100644
index f22693a5cc..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
+++ /dev/null
@@ -1,324 +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;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Random;
-
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hyracks.api.client.NodeControllerInfo;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class IndexingScheduler {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    /** a list of NCs */
-    private String[] NCs;
-
-    /** a map from ip to NCs */
-    private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
-
-    /** a map from the NC name to the index */
-    private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
-
-    /**
-     * The constructor of the scheduler.
-     *
-     * @param ncNameToNcInfos
-     * @throws HyracksException
-     */
-    public IndexingScheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
-        try {
-            loadIPAddressToNCMap(ncNameToNcInfos);
-        } catch (Exception e) {
-            throw HyracksException.create(e);
-        }
-    }
-
-    /**
-     * Set location constraints for a file scan operator with a list of file
-     * splits. It tries to assign splits to their local machines fairly
-     * Locality is more important than fairness
-     *
-     * @throws HyracksDataException
-     */
-    public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
-        if (splits == null) {
-            /* deal the case when the splits array is null */
-            return new String[] {};
-        }
-        int[] workloads = new int[NCs.length];
-        Arrays.fill(workloads, 0);
-        String[] locations = new String[splits.length];
-        Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
-        /*
-         * upper bound is number of splits
-         */
-        int upperBoundSlots = splits.length;
-
-        try {
-            Random random = new Random(System.currentTimeMillis());
-            boolean scheduled[] = new boolean[splits.length];
-            Arrays.fill(scheduled, false);
-            /*
-             * scan the splits and build the popularity map
-             * give the machines with less local splits more scheduling priority
-             */
-            buildPopularityMap(splits, locationToNumOfSplits);
-            HashMap<String, Integer> locationToNumOfAssignement = new HashMap<String, Integer>();
-            for (String location : locationToNumOfSplits.keySet()) {
-                locationToNumOfAssignement.put(location, 0);
-            }
-            /*
-             * push data-local upper-bounds slots to each machine
-             */
-            scheduleLocalSlots(splits, workloads, locations, upperBoundSlots, random, scheduled, locationToNumOfSplits,
-                    locationToNumOfAssignement);
-
-            int dataLocalCount = 0;
-            for (int i = 0; i < scheduled.length; i++) {
-                if (scheduled[i] == true) {
-                    dataLocalCount++;
-                }
-            }
-            LOGGER.info("Data local rate: "
-                    + (scheduled.length == 0 ? 0.0 : ((float) dataLocalCount / (float) (scheduled.length))));
-            /*
-             * push non-data-local upper-bounds slots to each machine
-             */
-            locationToNumOfAssignement.clear();
-            for (String nc : NCs) {
-                locationToNumOfAssignement.put(nc, 0);
-            }
-            for (int i = 0; i < scheduled.length; i++) {
-                if (scheduled[i]) {
-                    locationToNumOfAssignement.put(locations[i], locationToNumOfAssignement.get(locations[i]) + 1);
-                }
-            }
-
-            scheduleNonLocalSlots(splits, workloads, locations, upperBoundSlots, scheduled, locationToNumOfAssignement);
-            return locations;
-        } catch (IOException e) {
-            throw HyracksException.create(e);
-        }
-    }
-
-    /**
-     * Schedule non-local slots to each machine
-     *
-     * @param splits
-     *            The HDFS file splits.
-     * @param workloads
-     *            The current capacity of each machine.
-     * @param locations
-     *            The result schedule.
-     * @param slotLimit
-     *            The maximum slots of each machine.
-     * @param scheduled
-     *            Indicate which slot is scheduled.
-     * @param locationToNumOfAssignement
-     */
-    private void scheduleNonLocalSlots(InputSplit[] splits, final int[] workloads, String[] locations, int slotLimit,
-            boolean[] scheduled, final HashMap<String, Integer> locationToNumOfAssignement)
-            throws IOException, UnknownHostException {
-
-        PriorityQueue<String> scheduleCadndiates =
-                new PriorityQueue<String>(NCs.length, Comparator.comparing(locationToNumOfAssignement::get));
-
-        scheduleCadndiates.addAll(Arrays.asList(NCs));
-        /*
-         * schedule no-local file reads
-         */
-        for (int i = 0; i < splits.length; i++) {
-            /* if there is no data-local NC choice, choose a random one */
-            if (!scheduled[i]) {
-                String selectedNcName = scheduleCadndiates.remove();
-                if (selectedNcName != null) {
-                    int ncIndex = ncNameToIndex.get(selectedNcName);
-                    workloads[ncIndex]++;
-                    scheduled[i] = true;
-                    locations[i] = selectedNcName;
-                    locationToNumOfAssignement.put(selectedNcName, workloads[ncIndex]);
-                    scheduleCadndiates.add(selectedNcName);
-                }
-            }
-        }
-    }
-
-    /**
-     * Schedule data-local slots to each machine.
-     *
-     * @param splits
-     *            The HDFS file splits.
-     * @param workloads
-     *            The current capacity of each machine.
-     * @param locations
-     *            The result schedule.
-     * @param slots
-     *            The maximum slots of each machine.
-     * @param random
-     *            The random generator.
-     * @param scheduled
-     *            Indicate which slot is scheduled.
-     * @throws IOException
-     * @throws UnknownHostException
-     */
-    private void scheduleLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slots, Random random,
-            boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits,
-            final HashMap<String, Integer> locationToNumOfAssignement) throws IOException, UnknownHostException {
-        /* scheduling candidates will be ordered inversely according to their popularity */
-        PriorityQueue<String> scheduleCadndiates = new PriorityQueue<>(3, (s1, s2) -> {
-            int assignmentDifference = locationToNumOfAssignement.get(s1).compareTo(locationToNumOfAssignement.get(s2));
-            if (assignmentDifference != 0) {
-                return assignmentDifference;
-            }
-            return locationToNumSplits.get(s1).compareTo(locationToNumSplits.get(s2));
-        });
-
-        for (int i = 0; i < splits.length; i++) {
-            if (scheduled[i]) {
-                continue;
-            }
-            /*
-             * get the location of all the splits
-             */
-            String[] locs = splits[i].getLocations();
-            if (locs.length > 0) {
-                scheduleCadndiates.clear();
-                Collections.addAll(scheduleCadndiates, locs);
-
-                for (String candidate : scheduleCadndiates) {
-                    /*
-                     * get all the IP addresses from the name
-                     */
-                    InetAddress[] allIps = InetAddress.getAllByName(candidate);
-                    /*
-                     * iterate overa all ips
-                     */
-                    for (InetAddress ip : allIps) {
-                        /*
-                         * if the node controller exists
-                         */
-                        if (ipToNcMapping.get(ip.getHostAddress()) != null) {
-                            /*
-                             * set the ncs
-                             */
-                            List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
-                            int arrayPos = random.nextInt(dataLocations.size());
-                            String nc = dataLocations.get(arrayPos);
-                            int pos = ncNameToIndex.get(nc);
-                            /*
-                             * check if the node is already full
-                             */
-                            if (workloads[pos] < slots) {
-                                locations[i] = nc;
-                                workloads[pos]++;
-                                scheduled[i] = true;
-                                locationToNumOfAssignement.put(candidate,
-                                        locationToNumOfAssignement.get(candidate) + 1);
-                                break;
-                            }
-                        }
-                    }
-                    /*
-                     * break the loop for data-locations if the schedule has
-                     * already been found
-                     */
-                    if (scheduled[i]) {
-                        break;
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Scan the splits once and build a popularity map
-     *
-     * @param splits
-     *            the split array
-     * @param locationToNumOfSplits
-     *            the map to be built
-     * @throws IOException
-     */
-    private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
-            throws IOException {
-        for (InputSplit split : splits) {
-            String[] locations = split.getLocations();
-            for (String loc : locations) {
-                IntWritable locCount = locationToNumOfSplits.get(loc);
-                if (locCount == null) {
-                    locCount = new IntWritable(0);
-                    locationToNumOfSplits.put(loc, locCount);
-                }
-                locCount.set(locCount.get() + 1);
-            }
-        }
-    }
-
-    /**
-     * Load the IP-address-to-NC map from the NCNameToNCInfoMap
-     *
-     * @param ncNameToNcInfos
-     * @throws HyracksException
-     */
-    private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
-        try {
-            NCs = new String[ncNameToNcInfos.size()];
-            ipToNcMapping.clear();
-            ncNameToIndex.clear();
-            int i = 0;
-
-            /*
-             * build the IP address to NC map
-             */
-            for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
-                String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().lookupIpAddress())
-                        .getHostAddress();
-                List<String> matchedNCs = ipToNcMapping.computeIfAbsent(ipAddr, k -> new ArrayList<>());
-                matchedNCs.add(entry.getKey());
-                NCs[i] = entry.getKey();
-                i++;
-            }
-
-            /*
-             * set up the NC name to index mapping
-             */
-            for (i = 0; i < NCs.length; i++) {
-                ncNameToIndex.put(NCs[i], i);
-            }
-        } catch (Exception e) {
-            throw HyracksException.create(e);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java
deleted file mode 100644
index 07d09dbc2e..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RCRecordIdReader.java
+++ /dev/null
@@ -1,43 +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;
-
-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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
deleted file mode 100644
index 6cecc20bf2..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
+++ /dev/null
@@ -1,94 +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;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.api.IExternalIndexer;
-import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-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.api.exceptions.HyracksDataException;
-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 final AMutableInt32 fileNumber = new AMutableInt32(0);
-    protected final AMutableInt64 offset = new AMutableInt64(0);
-    protected long nextOffset;
-    protected final AMutableInt32 rowNumber = new AMutableInt32(0);
-    protected RecordReader<?, Writable> recordReader;
-
-    @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<IAObject> intSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-    @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<IAObject> longSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-
-    @Override
-    public void reset(IIndexingDatasource reader) throws HyracksDataException {
-        try {
-            //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);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public void index(ArrayTupleBuilder tb) throws HyracksDataException {
-        try {
-            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);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public int getNumberOfFields() {
-        return NUM_OF_FIELDS;
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.java
deleted file mode 100644
index 9027101309..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordId.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.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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.java
deleted file mode 100644
index 3cca7e10d5..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReader.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.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 MISSING_BYTE = ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
-    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) == MISSING_BYTE) {
-            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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java
deleted file mode 100644
index d0bf2ff1c0..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordIdReaderFactory.java
+++ /dev/null
@@ -1,38 +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;
-
-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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index b190e33697..596edb2801 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -37,13 +37,11 @@ import org.apache.asterix.external.api.IIndexibleExternalDataSource;
 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.IndexingStreamRecordReader;
 import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
 import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetFileRecordReader;
 import org.apache.asterix.external.input.record.reader.stream.StreamRecordReader;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
-import org.apache.asterix.external.provider.ExternalIndexerProvider;
 import org.apache.asterix.external.provider.StreamRecordReaderProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
@@ -77,7 +75,6 @@ public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IInd
     protected ConfFactory confFactory;
     protected boolean configured = false;
     protected static Scheduler hdfsScheduler;
-    protected static IndexingScheduler indexingScheduler;
     protected static Boolean initialized = false;
     protected static Object initLock = new Object();
     protected List<ExternalFile> files;
@@ -117,11 +114,7 @@ public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IInd
             } else {
                 inputSplits = HDFSUtils.getSplits(conf, files);
             }
-            if (indexingOp) {
-                readSchedule = indexingScheduler.getLocationConstraints(inputSplits);
-            } else {
-                readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
-            }
+            readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
             inputSplitsFactory = new InputSplitsFactory(inputSplits);
             read = new boolean[readSchedule.length];
             Arrays.fill(read, false);
@@ -214,7 +207,6 @@ public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IInd
             synchronized (initLock) {
                 if (!initialized) {
                     hdfsScheduler = HDFSUtils.initializeHDFSScheduler(serviceCtx);
-                    indexingScheduler = HDFSUtils.initializeIndexingHDFSScheduler(serviceCtx);
                     initialized = true;
                 }
             }
@@ -243,7 +235,7 @@ public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IInd
     public IRecordReader<? extends Object> createRecordReader(IHyracksTaskContext ctx, int partition)
             throws HyracksDataException {
         try {
-            IExternalIndexer indexer = files == null ? null : ExternalIndexerProvider.getIndexer(configuration);
+            IExternalIndexer indexer = null;
             if (recordReaderClazz != null) {
                 StreamRecordReader streamReader = (StreamRecordReader) recordReaderClazz.getConstructor().newInstance();
                 streamReader.configure(ctx, createInputStream(ctx, partition, indexer), configuration);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java
deleted file mode 100644
index 6c7c42e4b2..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java
+++ /dev/null
@@ -1,79 +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.input.record.reader.hdfs;
-
-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() throws IOException {
-        reusableByteBuffer.clear();
-        if (reusableByteBuffer.remaining() < value.getLength()) {
-            reusableByteBuffer = ByteBuffer
-                    .allocateDirect((int) (value.getLength() * ExternalDataConstants.DEFAULT_BUFFER_INCREMENT_FACTOR));
-        }
-        reusableByteBuffer.put(value.getBytes(), 0, value.getLength());
-        reusableByteBuffer.flip();
-        while (reusableByteBuffer.hasRemaining()) {
-            reusableCharBuffer.clear();
-            decoder.decode(reusableByteBuffer, reusableCharBuffer, false);
-            reusableCharBuffer.flip();
-            record.append(reusableCharBuffer);
-        }
-        record.endRecord();
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java
deleted file mode 100644
index 28abddb4a0..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java
+++ /dev/null
@@ -1,113 +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.input.record.reader.hdfs;
-
-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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
deleted file mode 100644
index 75d431d9ad..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.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.input.record.reader.hdfs;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.common.api.IApplicationContext;
-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.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.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.hdfs.dataflow.ConfFactory;
-
-public class HDFSLookupReaderFactory<T> implements ILookupReaderFactory<T> {
-
-    private static final long serialVersionUID = 1L;
-    protected ConfFactory confFactory;
-    protected Map<String, String> configuration;
-    protected transient AlgebricksAbsolutePartitionConstraint clusterLocations;
-    protected transient IServiceContext serviceCtx;
-
-    public HDFSLookupReaderFactory() {
-    }
-
-    @Override
-    public DataSourceType getDataSourceType() {
-        return DataSourceType.RECORDS;
-    }
-
-    @Override
-    public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AsterixException {
-        clusterLocations = HDFSUtils.getPartitionConstraints((IApplicationContext) serviceCtx.getApplicationContext(),
-                clusterLocations);
-        return clusterLocations;
-    }
-
-    @Override
-    public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
-            IWarningCollector warningCollector) throws AsterixException {
-        this.serviceCtx = serviceCtx;
-        this.configuration = configuration;
-        JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
-        try {
-            confFactory = new ConfFactory(conf);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
-
-    }
-
-    @Override
-    public boolean isIndexible() {
-        return false;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
-            ExternalFileIndexAccessor snapshotAccessor) throws HyracksDataException {
-        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
-        JobConf conf = confFactory.getConf();
-        FileSystem fs;
-        try {
-            fs = FileSystem.get(conf);
-        } catch (IOException e) {
-            throw new HyracksDataException("Unable to get filesystem object", e);
-        }
-        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);
-            default:
-                throw new HyracksDataException("Unrecognised input format: " + inputFormatParameter);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java
deleted file mode 100644
index 46ae9ac194..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java
+++ /dev/null
@@ -1,72 +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.input.record.reader.hdfs;
-
-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.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class SequenceLookupReader extends AbstractCharRecordLookupReader {
-
-    public SequenceLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
deleted file mode 100644
index 199d05a5d7..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.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.input.record.reader.hdfs;
-
-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.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class TextLookupReader extends AbstractCharRecordLookupReader {
-
-    public TextLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    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/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
deleted file mode 100644
index 4a7635c088..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
+++ /dev/null
@@ -1,96 +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.operators;
-
-import java.util.List;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-
-// This is an operator that takes a single file index and an array of secondary indexes
-// it is intended to be used for
-// 1. commit transaction operation
-// 2. abort transaction operation
-// 3. recover transaction operation
-public abstract class AbstractExternalDatasetIndexesOperatorDescriptor
-        extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private List<IIndexDataflowHelperFactory> treeIndexesDataflowHelperFactories;
-
-    public AbstractExternalDatasetIndexesOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            List<IIndexDataflowHelperFactory> treeIndexesDataflowHelperFactories) {
-        super(spec, 0, 0);
-        this.treeIndexesDataflowHelperFactories = treeIndexesDataflowHelperFactories;
-    }
-
-    // opening and closing the index is done inside these methods since we don't always need open indexes
-    protected abstract void performOpOnIndex(IIndexDataflowHelper indexDataflowHelper, IHyracksTaskContext ctx)
-            throws HyracksDataException;
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-        return new AbstractOperatorNodePushable() {
-
-            @Override
-            public void initialize() throws HyracksDataException {
-                try {
-                    // perform operation on btrees
-                    for (int i = 0; i < treeIndexesDataflowHelperFactories.size(); i++) {
-                        IIndexDataflowHelper indexHelper = treeIndexesDataflowHelperFactories.get(i)
-                                .create(ctx.getJobletContext().getServiceContext(), partition);
-                        performOpOnIndex(indexHelper, ctx);
-                    }
-                } catch (Exception e) {
-                    throw HyracksDataException.create(e);
-                }
-            }
-
-            @Override
-            public void deinitialize() throws HyracksDataException {
-            }
-
-            @Override
-            public int getInputArity() {
-                return 0;
-            }
-
-            @Override
-            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
-                    throws HyracksDataException {
-            }
-
-            @Override
-            public IFrameWriter getInputFrameWriter(int index) {
-                return null;
-            }
-
-        };
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
deleted file mode 100644
index 56bedeadda..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
+++ /dev/null
@@ -1,56 +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.operators;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-
-public class ExternalBTreeSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private final int version;
-
-    public ExternalBTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
-            int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
-            IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
-            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
-        super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false, null);
-        this.version = version;
-    }
-
-    @Override
-    public ExternalBTreeSearchOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new ExternalBTreeSearchOperatorNodePushable(ctx, partition,
-                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
-                lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, version);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
deleted file mode 100644
index 4e669c32d8..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ /dev/null
@@ -1,94 +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.operators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-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;
-import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorNodePushable;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-
-public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
-
-    private final int version;
-
-    public ExternalBTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition,
-            RecordDescriptor intputRecDesc, int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
-            boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput,
-            boolean retainMissing, IMissingWriterFactory missingWriterFactory,
-            ISearchOperationCallbackFactory searchCallbackFactory, int version) throws HyracksDataException {
-        super(ctx, partition, intputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
-                indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false,
-                null);
-        this.version = version;
-    }
-
-    // We override the open function to search a specific version of the index
-    @Override
-    public void open() throws HyracksDataException {
-        writer.open();
-        accessor = new FrameTupleAccessor(inputRecDesc);
-        indexHelper.open();
-        index = indexHelper.getIndexInstance();
-        if (retainMissing) {
-            int fieldCount = getFieldCount();
-            nonMatchTupleBuild = new ArrayTupleBuilder(fieldCount);
-            DataOutput out = nonMatchTupleBuild.getDataOutput();
-            for (int i = 0; i < fieldCount; i++) {
-                try {
-                    nonMatchWriter.writeMissing(out);
-                } catch (IOException e) {
-                    e.printStackTrace();
-                }
-                nonMatchTupleBuild.addFieldEndOffset();
-            }
-        } else {
-            nonMatchTupleBuild = null;
-        }
-        ExternalBTreeWithBuddy externalIndex = (ExternalBTreeWithBuddy) index;
-        try {
-            searchPred = createSearchPredicate();
-            tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
-            dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(new VSizeFrame(ctx));
-            ISearchOperationCallback searchCallback =
-                    searchCallbackFactory.createSearchOperationCallback(indexHelper.getResource().getId(), ctx, null);
-            // The next line is the reason we override this method
-            indexAccessor = externalIndex.createAccessor(searchCallback, version);
-            cursor = createCursor();
-            if (retainInput) {
-                frameTuple = new FrameTupleReference();
-            }
-        } catch (Throwable th) {
-            throw HyracksDataException.create(th);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
deleted file mode 100644
index 20f0c55293..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesAbortOperatorDescriptor.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.operators;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
-
-public class ExternalDatasetIndexesAbortOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalDatasetIndexesAbortOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories) {
-        super(spec, indexesDataflowHelperFactories);
-    }
-
-    @Override
-    protected void performOpOnIndex(IIndexDataflowHelper indexDataflowHelper, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        String path = indexDataflowHelper.getResource().getPath();
-        IIOManager ioManager = ctx.getIoManager();
-        FileReference file = ioManager.resolve(path);
-        AbortRecoverLSMIndexFileManager fileManager = new AbortRecoverLSMIndexFileManager(ctx.getIoManager(), file);
-        fileManager.deleteTransactionFiles();
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
deleted file mode 100644
index 337946b1e9..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
+++ /dev/null
@@ -1,57 +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.operators;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ExternalDatasetIndexesCommitOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    public ExternalDatasetIndexesCommitOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories) {
-        super(spec, indexesDataflowHelperFactories);
-    }
-
-    @Override
-    protected void performOpOnIndex(IIndexDataflowHelper indexHelper, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        String path = indexHelper.getResource().getPath();
-        IIOManager ioManager = ctx.getIoManager();
-        FileReference file = ioManager.resolve(path);
-        LOGGER.warn("performing the operation on " + file.getFile().getAbsolutePath());
-        // Get index
-        IIndex index = indexHelper.getIndexInstance();
-        // commit transaction
-        ((ITwoPCIndex) index).commitTransaction();
-        LOGGER.warn("operation on " + file.getFile().getAbsolutePath() + " Succeded");
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
deleted file mode 100644
index 5413e4d07d..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.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.external.operators;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
-
-public class ExternalDatasetIndexesRecoverOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalDatasetIndexesRecoverOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories) {
-        super(spec, indexesDataflowHelperFactories);
-    }
-
-    @Override
-    protected void performOpOnIndex(IIndexDataflowHelper indexDataflowHelper, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        String path = indexDataflowHelper.getResource().getPath();
-        IIOManager ioManager = ctx.getIoManager();
-        FileReference file = ioManager.resolve(path);
-        AbortRecoverLSMIndexFileManager fileManager = new AbortRecoverLSMIndexFileManager(ctx.getIoManager(), file);
-        fileManager.recoverTransaction();
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
deleted file mode 100644
index df4c093ccc..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
+++ /dev/null
@@ -1,119 +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.operators;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.common.api.IIndexBuilder;
-import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-
-/**
- * For the replicated file index
- * It creates and bulkloads initial set of files
- */
-public class ExternalFilesIndexCreateOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private List<ExternalFile> files;
-    private IIndexDataflowHelperFactory dataflowHelperFactory;
-    private IIndexBuilderFactory indexBuilderFactory;
-
-    public ExternalFilesIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IIndexBuilderFactory indexBuilderFactory, IIndexDataflowHelperFactory dataflowHelperFactory,
-            List<ExternalFile> files) {
-        super(spec, 0, 0);
-        this.indexBuilderFactory = indexBuilderFactory;
-        this.dataflowHelperFactory = dataflowHelperFactory;
-        this.files = files;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new AbstractOperatorNodePushable() {
-            @Override
-            public void initialize() throws HyracksDataException {
-                IIndexBuilder indexBuilder = indexBuilderFactory.create(ctx, partition);
-                IIndexDataflowHelper indexHelper =
-                        dataflowHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
-                FileIndexTupleTranslator filesTupleTranslator = new FileIndexTupleTranslator();
-                // Build the index
-                indexBuilder.build();
-                // Open the index
-                indexHelper.open();
-                try {
-                    ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
-                    Map<String, Object> parameters = new HashMap<>();
-                    parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
-                            LSMComponentId.DEFAULT_COMPONENT_ID);
-                    // Create bulk loader
-                    IIndexBulkLoader bulkLoader =
-                            index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false, parameters);
-                    // Load files
-                    for (ExternalFile file : files) {
-                        bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
-                    }
-                    bulkLoader.end();
-                } finally {
-                    indexHelper.close();
-                }
-            }
-
-            @Override
-            public void deinitialize() throws HyracksDataException {
-            }
-
-            @Override
-            public int getInputArity() {
-                return 0;
-            }
-
-            @Override
-            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
-                    throws HyracksDataException {
-            }
-
-            @Override
-            public IFrameWriter getInputFrameWriter(int index) {
-                return null;
-            }
-
-        };
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
deleted file mode 100644
index 477f0fec60..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
+++ /dev/null
@@ -1,129 +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.operators;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree.LSMTwoPCBTreeBulkLoader;
-import org.apache.hyracks.storage.common.IIndex;
-
-/**
- * This operator is intended solely for external dataset files replicated index.
- * It bulkmodify the index creating a hidden transaction component which later might be committed or deleted
- */
-public class ExternalFilesIndexModificationOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private List<ExternalFile> files;
-    private IIndexDataflowHelperFactory dataflowHelperFactory;
-
-    public ExternalFilesIndexModificationOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IIndexDataflowHelperFactory dataflowHelperFactory, List<ExternalFile> files) {
-        super(spec, 0, 0);
-        this.dataflowHelperFactory = dataflowHelperFactory;
-        this.files = files;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new AbstractOperatorNodePushable() {
-            @Override
-            public void initialize() throws HyracksDataException {
-                final IIndexDataflowHelper indexHelper =
-                        dataflowHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
-                FileIndexTupleTranslator filesTupleTranslator = new FileIndexTupleTranslator();
-                // Open and get
-                indexHelper.open();
-                IIndex index = indexHelper.getIndexInstance();
-                LSMTwoPCBTreeBulkLoader bulkLoader = null;
-                Map<String, Object> parameters = new HashMap<>();
-                try {
-                    bulkLoader = (LSMTwoPCBTreeBulkLoader) ((ExternalBTree) index)
-                            .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), parameters);
-                    // Load files
-                    // The files must be ordered according to their numbers
-                    for (ExternalFile file : files) {
-                        switch (file.getPendingOp()) {
-                            case ADD_OP:
-                            case APPEND_OP:
-                                bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
-                                break;
-                            case DROP_OP:
-                                bulkLoader.delete(filesTupleTranslator.getTupleFromFile(file));
-                                break;
-                            case NO_OP:
-                                break;
-                            default:
-                                throw RuntimeDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP, sourceLoc,
-                                        file.getPendingOp());
-                        }
-                    }
-                    bulkLoader.end();
-                } catch (Exception e) {
-                    if (bulkLoader != null) {
-                        bulkLoader.abort();
-                    }
-                    throw HyracksDataException.create(e);
-                } finally {
-                    indexHelper.close();
-                }
-            }
-
-            @Override
-            public void deinitialize() throws HyracksDataException {
-            }
-
-            @Override
-            public int getInputArity() {
-                return 0;
-            }
-
-            @Override
-            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
-                    throws HyracksDataException {
-            }
-
-            @Override
-            public IFrameWriter getInputFrameWriter(int index) {
-                return null;
-            }
-
-        };
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorDescriptor.java
deleted file mode 100644
index f910fb4ec8..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorDescriptor.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.operators;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
-
-public class ExternalIndexBulkLoadOperatorDescriptor extends TreeIndexBulkLoadOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private final int version;
-    private final ITupleFilterFactory tupleFilterFactory;
-
-    public ExternalIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
-            int[] fieldPermutation, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, IIndexDataflowHelperFactory indexHelperFactory, int version,
-            ITupleFilterFactory tupleFilterFactory) {
-        super(spec, outRecDesc, fieldPermutation, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                indexHelperFactory);
-        this.version = version;
-        this.tupleFilterFactory = tupleFilterFactory;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new ExternalIndexBulkLoadOperatorNodePushable(indexHelperFactory, ctx, partition, fieldPermutation,
-                fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                recordDescProvider.getInputRecordDescriptor(this.getActivityId(), 0), version, tupleFilterFactory);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
deleted file mode 100644
index 39f23ed595..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
+++ /dev/null
@@ -1,61 +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.operators;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
-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.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-
-public class ExternalIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
-
-    private final int version;
-
-    public ExternalIndexBulkLoadOperatorNodePushable(IIndexDataflowHelperFactory indexDataflowHelperFactory,
-            IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, RecordDescriptor recDesc, int version,
-            ITupleFilterFactory tupleFilterFactory) throws HyracksDataException {
-        super(indexDataflowHelperFactory, ctx, partition, fieldPermutation, fillFactor, verifyInput, numElementsHint,
-                checkIfEmptyIndex, recDesc, tupleFilterFactory);
-        this.version = version;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        super.open();
-        ((ITwoPCIndex) index).setCurrentVersion(version);
-    }
-
-    @Override
-    protected void initializeBulkLoader() throws HyracksDataException {
-        Map<String, Object> parameters = new HashMap<>();
-        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
-        bulkLoader = ((ILSMIndex) index).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                parameters);
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java
deleted file mode 100644
index 674ee42209..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorDescriptor.java
+++ /dev/null
@@ -1,53 +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.operators;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
-
-public class ExternalIndexBulkModifyOperatorDescriptor extends TreeIndexBulkLoadOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private final int[] deletedFiles;
-    private final ITupleFilterFactory tupleFilterFactory;
-
-    public ExternalIndexBulkModifyOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IIndexDataflowHelperFactory dataflowHelperFactory, int[] deletedFiles, int[] fieldPermutation,
-            float fillFactor, boolean verifyInput, long numElementsHint, ITupleFilterFactory tupleFilterFactory) {
-        super(spec, null, fieldPermutation, fillFactor, verifyInput, numElementsHint, false, dataflowHelperFactory,
-                tupleFilterFactory);
-        this.deletedFiles = deletedFiles;
-        this.tupleFilterFactory = tupleFilterFactory;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new ExternalIndexBulkModifyOperatorNodePushable(indexHelperFactory, ctx, partition, fieldPermutation,
-                fillFactor, verifyInput, numElementsHint,
-                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), deletedFiles, tupleFilterFactory);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
deleted file mode 100644
index 745853e418..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.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.operators;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.asterix.external.indexing.FilesIndexDescription;
-import org.apache.asterix.om.base.AMutableInt32;
-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;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-
-public class ExternalIndexBulkModifyOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
-
-    private final int[] deletedFiles;
-    private ArrayTupleBuilder buddyBTreeTupleBuilder =
-            new ArrayTupleBuilder(FilesIndexDescription.FILE_BUDDY_BTREE_RECORD_DESCRIPTOR.getFieldCount());
-    private AMutableInt32 fileNumber = new AMutableInt32(0);
-    private ArrayTupleReference deleteTuple = new ArrayTupleReference();
-
-    public ExternalIndexBulkModifyOperatorNodePushable(IIndexDataflowHelperFactory indexHelperFactory,
-            IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float fillFactor, boolean verifyInput,
-            long numElementsHint, RecordDescriptor inputRecDesc, int[] deletedFiles,
-            ITupleFilterFactory tupleFilterFactory) throws HyracksDataException {
-        super(indexHelperFactory, ctx, partition, fieldPermutation, fillFactor, verifyInput, numElementsHint, false,
-                inputRecDesc, tupleFilterFactory);
-        this.deletedFiles = deletedFiles;
-    }
-
-    // We override this method to do two things
-    // when creating the bulkLoader, it creates a transaction bulk loader
-    // It uses the bulkLoader to insert delete tuples for the deleted files
-    @Override
-    public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(recDesc);
-        indexHelper.open();
-        index = indexHelper.getIndexInstance();
-        try {
-            writer.open();
-            // Transactional BulkLoader
-            Map<String, Object> parameters = new HashMap<>();
-            bulkLoader = ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length,
-                    parameters);
-            // Delete files
-            for (int i = 0; i < deletedFiles.length; i++) {
-                fileNumber.setValue(deletedFiles[i]);
-                FilesIndexDescription.getBuddyBTreeTupleFromFileNumber(deleteTuple, buddyBTreeTupleBuilder, fileNumber);
-                ((ITwoPCIndexBulkLoader) bulkLoader).delete(deleteTuple);
-            }
-        } catch (Throwable e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-            tuple.reset(accessor, i);
-            bulkLoader.add(tuple);
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (index != null) {
-            try {
-                bulkLoader.end();
-            } catch (Throwable th) {
-                throw HyracksDataException.create(th);
-            } finally {
-                try {
-                    indexHelper.close();
-                } finally {
-                    writer.close();
-                }
-            }
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        if (index != null) {
-            try {
-                ((ITwoPCIndexBulkLoader) bulkLoader).abort();
-            } finally {
-                writer.fail();
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
deleted file mode 100644
index e54b3d28a3..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
+++ /dev/null
@@ -1,133 +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.operators;
-
-import java.nio.ByteBuffer;
-
-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.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-
-/*
- * This operator is intended for using record ids to access data in external sources
- */
-public class ExternalLookupOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
-    private final LookupAdapterFactory<?> adapterFactory;
-    private final IIndexDataflowHelperFactory dataflowHelperFactory;
-    private final int version;
-    private final ISearchOperationCallbackFactory searchOpCallbackFactory;
-
-    public ExternalLookupOperatorDescriptor(IOperatorDescriptorRegistry spec, LookupAdapterFactory<?> adapterFactory,
-            RecordDescriptor outRecDesc, IIndexDataflowHelperFactory dataflowHelperFactory,
-            ISearchOperationCallbackFactory searchOpCallbackFactory, int version) {
-        super(spec, 1, 1);
-        outRecDescs[0] = outRecDesc;
-        this.dataflowHelperFactory = dataflowHelperFactory;
-        this.searchOpCallbackFactory = searchOpCallbackFactory;
-        this.version = version;
-        this.adapterFactory = adapterFactory;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
-            throws HyracksDataException {
-        // Create a file index accessor to be used for files lookup operations
-        final ExternalFileIndexAccessor snapshotAccessor = new ExternalFileIndexAccessor(
-                dataflowHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition),
-                searchOpCallbackFactory, version);
-        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
-            // The adapter that uses the file index along with the coming tuples to access files in HDFS
-            private LookupAdapter<?> adapter;
-            private boolean indexOpen = false;
-
-            @Override
-            public void open() throws HyracksDataException {
-                try {
-                    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) {
-                    throw HyracksDataException.create(th);
-                }
-            }
-
-            @Override
-            public void close() throws HyracksDataException {
-                HyracksDataException hde = null;
-                if (indexOpen) {
-                    try {
-                        snapshotAccessor.close();
-                    } catch (Throwable th) {
-                        hde = HyracksDataException.create(th);
-                    }
-                    try {
-                        adapter.close();
-                    } catch (Throwable th) {
-                        if (hde == null) {
-                            hde = HyracksDataException.create(th);
-                        } else {
-                            hde.addSuppressed(th);
-                        }
-                    }
-                }
-                if (hde != null) {
-                    throw hde;
-                }
-            }
-
-            @Override
-            public void fail() throws HyracksDataException {
-                try {
-                    adapter.fail();
-                } catch (Throwable th) {
-                    throw HyracksDataException.create(th);
-                }
-            }
-
-            @Override
-            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                try {
-                    adapter.nextFrame(buffer);
-                } catch (Throwable th) {
-                    throw HyracksDataException.create(th);
-                }
-            }
-
-            @Override
-            public void flush() throws HyracksDataException {
-                adapter.flush();
-            }
-        };
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
deleted file mode 100644
index bd6c922df5..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
+++ /dev/null
@@ -1,57 +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.operators;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
-
-public class ExternalRTreeSearchOperatorDescriptor extends RTreeSearchOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    private final int version;
-
-    public ExternalRTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
-            int[] keyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
-            IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
-            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
-        super(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
-                retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false, null);
-        this.version = version;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new ExternalRTreeSearchOperatorNodePushable(ctx, partition,
-                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), keyFields, minFilterFieldIndexes,
-                maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, version);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
deleted file mode 100644
index 508cff7290..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.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.operators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-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;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.impls.ExternalRTree;
-import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorNodePushable;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-
-public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperatorNodePushable {
-
-    private final int version;
-
-    public ExternalRTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition,
-            RecordDescriptor inputRecDesc, int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
-            IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
-            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            int version) throws HyracksDataException {
-        super(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false, null);
-        this.version = version;
-    }
-
-    // We override this method to specify the searched version of the index
-    @Override
-    public void open() throws HyracksDataException {
-        writer.open();
-        accessor = new FrameTupleAccessor(inputRecDesc);
-        indexHelper.open();
-        index = indexHelper.getIndexInstance();
-        if (retainMissing) {
-            int fieldCount = getFieldCount();
-            nonMatchTupleBuild = new ArrayTupleBuilder(fieldCount);
-            DataOutput out = nonMatchTupleBuild.getDataOutput();
-            for (int i = 0; i < fieldCount; i++) {
-                try {
-                    nonMatchWriter.writeMissing(out);
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
-                }
-                nonMatchTupleBuild.addFieldEndOffset();
-            }
-        } else {
-            nonMatchTupleBuild = null;
-        }
-        ExternalRTree rTreeIndex = (ExternalRTree) index;
-        try {
-            searchPred = createSearchPredicate();
-            tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
-            dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(new VSizeFrame(ctx));
-            ISearchOperationCallback searchCallback =
-                    searchCallbackFactory.createSearchOperationCallback(indexHelper.getResource().getId(), ctx, null);
-            // The next line is the reason we override this method...
-            // The right thing to do would be to change the signature of createAccessor
-            indexAccessor = rTreeIndex.createAccessor(searchCallback, version);
-            cursor = createCursor();
-            if (retainInput) {
-                frameTuple = new FrameTupleReference();
-            }
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
index a4fdcfbf06..4b0148e678 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
@@ -18,20 +18,14 @@
  */
 package org.apache.asterix.external.provider;
 
-import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
-import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
-import org.apache.asterix.external.api.IIndexingAdapterFactory;
 import org.apache.asterix.external.api.ITypedAdapterFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
@@ -57,29 +51,4 @@ public class AdapterFactoryProvider {
         adapterFactory.configure(serviceCtx, configuration, warningCollector);
         return adapterFactory;
     }
-
-    // get indexing adapter factory. this method has the side effect of modifying the configuration as necessary
-    public static IIndexingAdapterFactory getIndexingAdapterFactory(ICCServiceContext serviceCtx, String adapterName,
-            Map<String, String> configuration, ARecordType itemType, List<ExternalFile> snapshot, boolean indexingOp,
-            ARecordType metaType, IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
-        ExternalDataUtils.defaultConfiguration(configuration);
-        ExternalDataUtils.prepare(adapterName, configuration);
-        GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
-        adapterFactory.setOutputType(itemType);
-        adapterFactory.setMetaType(metaType);
-        adapterFactory.setSnapshot(snapshot, indexingOp);
-        adapterFactory.configure(serviceCtx, configuration, warningCollector);
-        return adapterFactory;
-    }
-
-    // Lookup Adapters
-    public static LookupAdapterFactory<?> getLookupAdapterFactory(ICCServiceContext serviceCtx,
-            Map<String, String> configuration, ARecordType recordType, int[] ridFields, boolean retainInput,
-            boolean retainMissing, IMissingWriterFactory missingWriterFactory, IWarningCollector warningCollector)
-            throws HyracksDataException, AlgebricksException {
-        LookupAdapterFactory<?> adapterFactory =
-                new LookupAdapterFactory<>(recordType, ridFields, retainInput, retainMissing, missingWriterFactory);
-        adapterFactory.configure(serviceCtx, configuration, warningCollector);
-        return adapterFactory;
-    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
deleted file mode 100644
index e24f43332e..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
+++ /dev/null
@@ -1,41 +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.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.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 {
-            throw new AsterixException("Unable to create indexer for data with format: " + inputFormatParameter);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
deleted file mode 100644
index db044793f2..0000000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.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.provider;
-
-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.hdfs.HDFSLookupReaderFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.HDFSUtils;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-
-public class LookupReaderFactoryProvider {
-
-    @SuppressWarnings("rawtypes")
-    public static ILookupReaderFactory getLookupReaderFactory(IServiceContext serviceCtx,
-            Map<String, String> configuration, IWarningCollector warningCollector) throws AsterixException {
-        String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
-        if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
-                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)) {
-            HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<>();
-            readerFactory.configure(serviceCtx, configuration, warningCollector);
-            return readerFactory;
-        } else {
-            throw new AsterixException("Unrecognized external format");
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index a4e47ca920..7bafd78471 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -35,8 +35,6 @@ import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 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.record.reader.hdfs.parquet.MapredParquetInputFormat;
 import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetReadSupport;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
@@ -56,7 +54,6 @@ import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -85,19 +82,6 @@ public class HDFSUtils {
         return scheduler;
     }
 
-    public static IndexingScheduler initializeIndexingHDFSScheduler(ICCServiceContext serviceCtx)
-            throws HyracksDataException {
-        IndexingScheduler scheduler = null;
-        try {
-            ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-            IHyracksClientConnection hcc = appCtx.getHcc();
-            scheduler = new IndexingScheduler(hcc.getNodeControllerInfos());
-        } catch (HyracksException e) {
-            throw new RuntimeDataException(ErrorCode.UTIL_HDFS_UTILS_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
@@ -271,17 +255,6 @@ public class HDFSUtils {
 
     }
 
-    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;
-            default:
-                return null;
-        }
-    }
-
     public static ARecordType getExpectedType(Configuration configuration) throws IOException {
         String encoded = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS, "");
         if (encoded.isEmpty() || encoded.equals(DataProjectionFiltrationInfo.ALL_FIELDS_TYPE.getTypeName())) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index cafa0016ab..91edc4c63c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -28,7 +28,6 @@ import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
@@ -50,8 +49,6 @@ import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.column.dataflow.LSMColumnBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResourceFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -93,19 +90,7 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
         ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         switch (dataset.getDatasetType()) {
             case EXTERNAL:
-                return index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))
-                        ? new ExternalBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories,
-                                filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
-                                ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
-                                typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
-                        : new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
-                                filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
-                                ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
-                                typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+                return null;
             case INTERNAL:
                 AsterixVirtualBufferCacheProvider vbcProvider =
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -157,7 +142,7 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
             return primaryTypeTraits;
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-            return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
+            return null;
         }
         Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
@@ -190,7 +175,7 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
             return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-            return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
+            return null;
         }
         Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
@@ -225,7 +210,7 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
         if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexType() != DatasetConfig.IndexType.SAMPLE) {
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
-                return FilesIndexDescription.BLOOM_FILTER_FIELDS;
+                return null;
             } else {
                 Index.ValueIndexDetails indexDetails = ((Index.ValueIndexDetails) index.getIndexDetails());
                 return new int[] { indexDetails.getKeyFieldNames().size() };
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 2d3e0e4ed3..409f0e73dc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -57,14 +57,10 @@ import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 import org.apache.asterix.external.adapter.factory.ExternalAdapterFactory;
-import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
 import org.apache.asterix.external.api.ITypedAdapterFactory;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
-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.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
 import org.apache.asterix.external.provider.AdapterFactoryProvider;
@@ -82,7 +78,6 @@ import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
-import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
@@ -92,7 +87,6 @@ import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.FullTextUtil;
 import org.apache.asterix.metadata.utils.IndexUtil;
@@ -625,10 +619,7 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
                             proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan,
                             tupleProjectorFactory);
         } else {
-            btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
-                    highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
-                    nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
-                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+            btreeSearchOp = null;
         }
         return new Pair<>(btreeSearchOp, spPc.second);
     }
@@ -689,10 +680,7 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
                     nonFilterWriterFactory, isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             // Create the operator
-            rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
-                    indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
-                    searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
-                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+            rtreeSearchOp = null;
         }
 
         return new Pair<>(rtreeSearchOp, spPc.second);
@@ -972,36 +960,7 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             JobSpecification jobSpec, Dataset dataset, int[] ridIndexes, boolean retainInput,
             IVariableTypeEnvironment typeEnv, IOperatorSchema opSchema, JobGenContext context,
             MetadataProvider metadataProvider, boolean retainMissing) throws AlgebricksException {
-        try {
-            // Get data type
-            ARecordType itemType =
-                    (ARecordType) MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
-                            dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
-            ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-            LookupAdapterFactory<?> adapterFactory = AdapterFactoryProvider.getLookupAdapterFactory(
-                    getApplicationContext().getServiceContext(), datasetDetails.getProperties(), itemType, ridIndexes,
-                    retainInput, retainMissing, context.getMissingWriterFactory(), context.getWarningCollector());
-            String fileIndexName = IndexingConstants.getFilesIndexName(dataset.getDatasetName());
-            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
-                    metadataProvider.getSplitProviderAndConstraints(dataset, fileIndexName);
-            Index fileIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-                    dataset.getDatasetName(), fileIndexName);
-            // Create the file index data flow helper
-            IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
-            // Create the out record descriptor, appContext and fileSplitProvider for the
-            // files index
-            RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-            ISearchOperationCallbackFactory searchOpCallbackFactory =
-                    dataset.getSearchCallbackFactory(storageComponentProvider, fileIndex, IndexOperation.SEARCH, null);
-            // Create the operator
-            ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
-                    outRecDesc, indexDataflowHelperFactory, searchOpCallbackFactory,
-                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
-            return new Pair<>(op, spPc.second);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        return null;
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 07749282e7..a35be40f9c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -19,6 +19,7 @@
 
 package org.apache.asterix.metadata.entities;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,7 +47,6 @@ import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -59,9 +59,7 @@ import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
 import org.apache.asterix.metadata.declared.ArrayBTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
-import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.InvertedIndexResourceFactoryProvider;
 import org.apache.asterix.metadata.utils.MetadataUtil;
@@ -387,42 +385,6 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             bActiveTxn.setValue(true);
             metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
         } else if (getDatasetType() == DatasetType.EXTERNAL) {
-            // External dataset
-            ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(this);
-            // #. prepare jobs to drop the datatset and the indexes in NC
-            List<Index> indexes =
-                    MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
-            for (int j = 0; j < indexes.size(); j++) {
-                if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
-                    jobsToExecute
-                            .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
-                } else {
-                    jobsToExecute.add(DatasetUtil.buildDropFilesIndexJobSpec(metadataProvider, this));
-                }
-            }
-
-            // #. mark the existing dataset as PendingDropOp
-            MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName, force);
-            MetadataManager.INSTANCE.addDataset(mdTxnCtx.getValue(),
-                    new Dataset(dataverseName, datasetName, getItemTypeDataverseName(), getItemTypeName(),
-                            getNodeGroupName(), getCompactionPolicy(), getCompactionPolicyProperties(),
-                            getDatasetDetails(), getHints(), getDatasetType(), getDatasetId(),
-                            MetadataUtil.PENDING_DROP_OP));
-
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
-            bActiveTxn.setValue(false);
-            progress.setValue(ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA);
-
-            // #. run the jobs
-            for (JobSpecification jobSpec : jobsToExecute) {
-                JobUtils.runJob(hcc, jobSpec, true);
-            }
-            if (!indexes.isEmpty()) {
-                ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(this);
-            }
-            mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
-            bActiveTxn.setValue(true);
-            metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
         }
 
         // #. finally, delete the dataset.
@@ -684,7 +646,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     @Override
     public List<List<String>> getPrimaryKeys() {
         if (getDatasetType() == DatasetType.EXTERNAL) {
-            return IndexingConstants.getRIDKeys(((ExternalDatasetDetails) getDatasetDetails()).getProperties());
+            return Collections.emptyList();
         }
         return ((InternalDatasetDetails) getDatasetDetails()).getPartitioningKey();
     }
@@ -710,7 +672,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             indicators = ((InternalDatasetDetails) getDatasetDetails()).getKeySourceIndicator();
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            IAType keyType = datasetType == DatasetType.EXTERNAL ? IndexingConstants.getFieldType(i)
+            IAType keyType = datasetType == DatasetType.EXTERNAL ? null
                     : (indicators == null || indicators.get(i) == 0)
                             ? recordType.getSubFieldType(partitioningKeys.get(i))
                             : metaType.getSubFieldType(partitioningKeys.get(i));
@@ -747,7 +709,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
         // Set the serde/traits for primary keys
         for (int i = 0; i < numPrimaryKeys; i++) {
             IAType keyType =
-                    datasetType == DatasetType.EXTERNAL ? IndexingConstants.getFieldType(i)
+                    datasetType == DatasetType.EXTERNAL ? null
                             : (indicators == null || indicators.get(i) == 0)
                                     ? itemType.getSubFieldType(partitioningKeys.get(i))
                                     : metaType.getSubFieldType(partitioningKeys.get(i));
@@ -786,7 +748,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             indicators = ((InternalDatasetDetails) getDatasetDetails()).getKeySourceIndicator();
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            IAType keyType = datasetType == DatasetType.EXTERNAL ? IndexingConstants.getFieldType(i)
+            IAType keyType = datasetType == DatasetType.EXTERNAL ? null
                     : (indicators == null || indicators.get(i) == 0)
                             ? recordType.getSubFieldType(partitioningKeys.get(i))
                             : metaType.getSubFieldType(partitioningKeys.get(i));
@@ -814,7 +776,7 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             indicators = ((InternalDatasetDetails) getDatasetDetails()).getKeySourceIndicator();
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            IAType keyType = datasetType == DatasetType.EXTERNAL ? IndexingConstants.getFieldType(i)
+            IAType keyType = datasetType == DatasetType.EXTERNAL ? null
                     : (indicators == null || indicators.get(i) == 0)
                             ? recordType.getSubFieldType(partitioningKeys.get(i))
                             : metaType.getSubFieldType(partitioningKeys.get(i));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
deleted file mode 100644
index c6eafe0cf4..0000000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ /dev/null
@@ -1,551 +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.utils;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.config.DatasetConfig.TransactionState;
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.external.api.ITypedAdapterFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.IndexingConstants;
-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.ExternalFilesIndexCreateOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalFilesIndexModificationOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
-import org.apache.asterix.external.provider.AdapterFactoryProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
-import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.utils.RuntimeUtils;
-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;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
-import org.apache.hyracks.storage.am.common.build.IndexBuilderFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.common.IResourceFactory;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ExternalIndexingOperations {
-    private static final Logger LOGGER = LogManager.getLogger();
-    /**
-     * Note: there's a special handling of this empty indexing field name in
-     * {@link org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator
-     * IndexTupleTranslator.createMetadataEntityFromARecord()}
-     */
-    public static final List<List<String>> FILE_INDEX_FIELD_NAMES =
-            Collections.unmodifiableList(Collections.singletonList(Collections.singletonList("")));
-    public static final List<IAType> FILE_INDEX_FIELD_TYPES =
-            Collections.unmodifiableList(Collections.singletonList(BuiltinType.ASTRING));
-
-    private ExternalIndexingOperations() {
-    }
-
-    public static boolean isIndexible(ExternalDatasetDetails ds) {
-        String adapter = ds.getAdapter();
-        if (adapter.equalsIgnoreCase(ExternalDataConstants.ALIAS_HDFS_ADAPTER)) {
-            return true;
-        }
-        return false;
-    }
-
-    public static boolean isRefereshActive(ExternalDatasetDetails ds) {
-        return ds.getState() != TransactionState.COMMIT;
-    }
-
-    public static boolean isValidIndexName(String datasetName, String indexName) {
-        return !datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX).equals(indexName);
-    }
-
-    public static int getRIDSize(Dataset dataset) {
-        ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        return IndexingConstants.getRIDSize(dsd.getProperties().get(IndexingConstants.KEY_INPUT_FORMAT));
-    }
-
-    public static IBinaryComparatorFactory[] getComparatorFactories(Dataset dataset) {
-        ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        return IndexingConstants.getComparatorFactories(dsd.getProperties().get(IndexingConstants.KEY_INPUT_FORMAT));
-    }
-
-    public static IBinaryComparatorFactory[] getBuddyBtreeComparatorFactories() {
-        return IndexingConstants.getBuddyBtreeComparatorFactories();
-    }
-
-    public static List<ExternalFile> getSnapshotFromExternalFileSystem(Dataset dataset) throws AlgebricksException {
-        ArrayList<ExternalFile> files = new ArrayList<>();
-        ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        try {
-            // Create the file system object
-            FileSystem fs = getFileSystemObject(datasetDetails.getProperties());
-            // Get paths of dataset
-            String path = datasetDetails.getProperties().get(ExternalDataConstants.KEY_PATH);
-            String[] paths = path.split(",");
-
-            // Add fileStatuses to files
-            for (String aPath : paths) {
-                FileStatus[] fileStatuses = fs.listStatus(new Path(aPath));
-                for (int i = 0; i < fileStatuses.length; i++) {
-                    int nextFileNumber = files.size();
-                    handleFile(dataset, files, fs, fileStatuses[i], nextFileNumber);
-                }
-            }
-            // Close file system
-            fs.close();
-            if (files.isEmpty()) {
-                throw new AlgebricksException("File Snapshot retrieved from external file system is empty");
-            }
-            return files;
-        } catch (Exception e) {
-            LOGGER.warn("Exception while trying to get snapshot from external system", e);
-            throw new AlgebricksException("Unable to get list of HDFS files " + e);
-        }
-    }
-
-    private static void handleFile(Dataset dataset, List<ExternalFile> files, FileSystem fs, FileStatus fileStatus,
-            int nextFileNumber) throws IOException {
-        if (fileStatus.isDirectory()) {
-            listSubFiles(dataset, fs, fileStatus, files);
-        } else {
-            files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), nextFileNumber,
-                    fileStatus.getPath().toUri().getPath(), new Date(fileStatus.getModificationTime()),
-                    fileStatus.getLen(), ExternalFilePendingOp.NO_OP));
-        }
-    }
-
-    /* list all files under the directory
-     * src is expected to be a folder
-     */
-    private static void listSubFiles(Dataset dataset, FileSystem srcFs, FileStatus src, List<ExternalFile> files)
-            throws IOException {
-        Path path = src.getPath();
-        FileStatus[] fileStatuses = srcFs.listStatus(path);
-        for (int i = 0; i < fileStatuses.length; i++) {
-            int nextFileNumber = files.size();
-            if (fileStatuses[i].isDirectory()) {
-                listSubFiles(dataset, srcFs, 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(), ExternalFilePendingOp.NO_OP));
-            }
-        }
-    }
-
-    public static FileSystem getFileSystemObject(Map<String, String> map) throws IOException {
-        Configuration conf = new Configuration();
-        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 buildFilesIndexCreateJobSpec(Dataset dataset,
-            List<ExternalFile> externalFilesSnapshot, MetadataProvider metadataProvider) throws AlgebricksException {
-        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
-                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyProperties = compactionInfo.second;
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .getSplitProviderAndConstraints(dataset, IndexingConstants.getFilesIndexName(dataset.getDatasetName()));
-        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-        String fileIndexName = IndexingConstants.getFilesIndexName(dataset.getDatasetName());
-        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
-                dataset.getDataverseName(), dataset.getDatasetName(), fileIndexName);
-        ARecordType recordType =
-                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-        IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, fileIndex, recordType, null,
-                mergePolicyFactory, mergePolicyProperties);
-        IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
-                secondaryFileSplitProvider, resourceFactory, true);
-        IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
-                storageComponentProvider.getStorageManager(), secondaryFileSplitProvider);
-        ExternalFilesIndexCreateOperatorDescriptor externalFilesOp = new ExternalFilesIndexCreateOperatorDescriptor(
-                spec, indexBuilderFactory, dataflowHelperFactory, externalFilesSnapshot);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, externalFilesOp,
-                secondarySplitsAndConstraint.second);
-        spec.addRoot(externalFilesOp);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    public static JobSpecification buildFilesIndexUpdateJobSpec(Dataset dataset,
-            List<ExternalFile> externalFilesSnapshot, MetadataProvider metadataProvider) throws AlgebricksException {
-        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .getSplitProviderAndConstraints(dataset, IndexingConstants.getFilesIndexName(dataset.getDatasetName()));
-        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-        IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
-                storageComponentProvider.getStorageManager(), secondaryFileSplitProvider);
-        ExternalFilesIndexModificationOperatorDescriptor externalFilesOp =
-                new ExternalFilesIndexModificationOperatorDescriptor(spec, dataflowHelperFactory,
-                        externalFilesSnapshot);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, externalFilesOp,
-                secondarySplitsAndConstraint.second);
-        spec.addRoot(externalFilesOp);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    /**
-     * This method create an indexing operator that index records in HDFS
-     *
-     * @param jobSpec
-     * @param itemType
-     * @param dataset
-     * @param files
-     * @param indexerDesc
-     * @param sourceLoc
-     * @return
-     * @throws AlgebricksException
-     * @throws HyracksDataException
-     * @throws Exception
-     */
-    private static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> getIndexingOperator(
-            MetadataProvider metadataProvider, JobSpecification jobSpec, IAType itemType, Dataset dataset,
-            List<ExternalFile> files, RecordDescriptor indexerDesc, SourceLocation sourceLoc)
-            throws HyracksDataException, AlgebricksException {
-        ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        Map<String, String> configuration = externalDatasetDetails.getProperties();
-        ITypedAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
-                metadataProvider.getApplicationContext().getServiceContext(), externalDatasetDetails.getAdapter(),
-                configuration, (ARecordType) itemType, files, true, null, NoOpWarningCollector.INSTANCE);
-        ExternalScanOperatorDescriptor scanOp =
-                new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory);
-        scanOp.setSourceLocation(sourceLoc);
-        return new Pair<>(scanOp, adapterFactory.getPartitionConstraint());
-    }
-
-    public static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
-            JobSpecification spec, MetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
-            RecordDescriptor indexerDesc, List<ExternalFile> files, SourceLocation sourceLoc)
-            throws HyracksDataException, AlgebricksException {
-        return getIndexingOperator(metadataProvider, spec, itemType, dataset,
-                files == null ? MetadataManager.INSTANCE
-                        .getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset) : files,
-                indexerDesc, sourceLoc);
-    }
-
-    /**
-     * At the end of this method, we expect to have 4 sets as follows:
-     * metadataFiles should contain only the files that are appended in their original state
-     * addedFiles should contain new files that has number assigned starting after the max original file number
-     * deletedFiles 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
-     * @param deletedFiles
-     * @param appendedFiles
-     * @return
-     * @throws AlgebricksException
-     */
-    public static boolean isDatasetUptodate(Dataset dataset, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> deletedFiles, List<ExternalFile> appendedFiles)
-            throws AlgebricksException {
-        boolean uptodate = true;
-        int newFileNumber = metadataFiles.get(metadataFiles.size() - 1).getFileNumber() + 1;
-
-        List<ExternalFile> fileSystemFiles = getSnapshotFromExternalFileSystem(dataset);
-
-        // Loop over file system files < taking care of added files >
-        for (ExternalFile fileSystemFile : fileSystemFiles) {
-            boolean fileFound = false;
-            Iterator<ExternalFile> mdFilesIterator = metadataFiles.iterator();
-            while (mdFilesIterator.hasNext()) {
-                ExternalFile metadataFile = mdFilesIterator.next();
-                if (!fileSystemFile.getFileName().equals(metadataFile.getFileName())) {
-                    continue;
-                }
-                // Same file name
-                if (fileSystemFile.getLastModefiedTime().equals(metadataFile.getLastModefiedTime())) {
-                    // Same timestamp
-                    if (fileSystemFile.getSize() == metadataFile.getSize()) {
-                        // Same size -> no op
-                        mdFilesIterator.remove();
-                        fileFound = true;
-                    } else {
-                        // Different size -> append op
-                        metadataFile.setPendingOp(ExternalFilePendingOp.APPEND_OP);
-                        fileSystemFile.setPendingOp(ExternalFilePendingOp.APPEND_OP);
-                        appendedFiles.add(fileSystemFile);
-                        fileFound = true;
-                        uptodate = false;
-                    }
-                } else {
-                    // Same file name, Different file mod date -> delete and add
-                    metadataFile.setPendingOp(ExternalFilePendingOp.DROP_OP);
-                    deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(), 0,
-                            metadataFile.getFileName(), metadataFile.getLastModefiedTime(), metadataFile.getSize(),
-                            ExternalFilePendingOp.DROP_OP));
-                    fileSystemFile.setPendingOp(ExternalFilePendingOp.ADD_OP);
-                    fileSystemFile.setFileNumber(newFileNumber);
-                    addedFiles.add(fileSystemFile);
-                    newFileNumber++;
-                    fileFound = true;
-                    uptodate = false;
-                }
-                if (fileFound) {
-                    break;
-                }
-            }
-            if (!fileFound) {
-                // File not stored previously in metadata -> pending add op
-                fileSystemFile.setPendingOp(ExternalFilePendingOp.ADD_OP);
-                fileSystemFile.setFileNumber(newFileNumber);
-                addedFiles.add(fileSystemFile);
-                newFileNumber++;
-                uptodate = false;
-            }
-        }
-
-        // Done with files from external file system -> metadata files now contain both deleted files and appended ones
-        // first, correct number assignment to deleted and updated files
-        for (ExternalFile deletedFile : deletedFiles) {
-            deletedFile.setFileNumber(newFileNumber);
-            newFileNumber++;
-        }
-        for (ExternalFile appendedFile : appendedFiles) {
-            appendedFile.setFileNumber(newFileNumber);
-            newFileNumber++;
-        }
-
-        // include the remaining deleted files
-        Iterator<ExternalFile> mdFilesIterator = metadataFiles.iterator();
-        while (mdFilesIterator.hasNext()) {
-            ExternalFile metadataFile = mdFilesIterator.next();
-            if (metadataFile.getPendingOp() == ExternalFilePendingOp.NO_OP) {
-                metadataFile.setPendingOp(ExternalFilePendingOp.DROP_OP);
-                deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(),
-                        newFileNumber, metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
-                        metadataFile.getSize(), metadataFile.getPendingOp()));
-                newFileNumber++;
-                uptodate = false;
-            }
-        }
-        return uptodate;
-    }
-
-    public static Dataset createTransactionDataset(Dataset dataset) {
-        ExternalDatasetDetails originalDsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        ExternalDatasetDetails dsd = new ExternalDatasetDetails(originalDsd.getAdapter(), originalDsd.getProperties(),
-                originalDsd.getTimestamp(), TransactionState.BEGIN);
-        return new Dataset(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName(), dataset.getNodeGroupName(), dataset.getCompactionPolicy(),
-                dataset.getCompactionPolicyProperties(), dsd, dataset.getHints(), DatasetType.EXTERNAL,
-                dataset.getDatasetId(), dataset.getPendingOp());
-    }
-
-    public static JobSpecification buildDropFilesIndexJobSpec(MetadataProvider metadataProvider, Dataset dataset)
-            throws AlgebricksException {
-        String indexName = IndexingConstants.getFilesIndexName(dataset.getDatasetName());
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                metadataProvider.getSplitProviderAndConstraints(dataset, indexName);
-        IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
-                metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
-        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
-                splitsAndConstraint.second);
-        spec.addRoot(btreeDrop);
-
-        return spec;
-    }
-
-    public static JobSpecification buildFilesIndexUpdateOp(Dataset ds, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider)
-            throws AlgebricksException {
-        ArrayList<ExternalFile> files = new ArrayList<>();
-        for (ExternalFile file : metadataFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
-                files.add(file);
-            } else if (file.getPendingOp() == ExternalFilePendingOp.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.NO_OP));
-                    }
-                }
-            }
-        }
-        for (ExternalFile file : addedFiles) {
-            files.add(file);
-        }
-        Collections.sort(files);
-        return buildFilesIndexUpdateJobSpec(ds, files, metadataProvider);
-    }
-
-    public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider,
-            SourceLocation sourceLoc) throws AlgebricksException {
-        // Create files list
-        ArrayList<ExternalFile> files = new ArrayList<>();
-
-        for (ExternalFile metadataFile : metadataFiles) {
-            if (metadataFile.getPendingOp() != ExternalFilePendingOp.APPEND_OP) {
-                files.add(metadataFile);
-            } else {
-                metadataFile.setPendingOp(ExternalFilePendingOp.NO_OP);
-                files.add(metadataFile);
-            }
-        }
-        // add new files
-        for (ExternalFile file : addedFiles) {
-            files.add(file);
-        }
-        // add appended files
-        for (ExternalFile file : appendedFiles) {
-            files.add(file);
-        }
-        return IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, files, sourceLoc);
-    }
-
-    public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException {
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager();
-        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
-        AlgebricksPartitionConstraint constraints = null;
-        for (Index index : indexes) {
-            IFileSplitProvider indexSplitProvider;
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints =
-                        metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName());
-                indexSplitProvider = sAndConstraints.first;
-                constraints = sAndConstraints.second;
-            } else {
-                indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds,
-                        IndexingConstants.getFilesIndexName(ds.getDatasetName())).first;
-            }
-            IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storageMgr, indexSplitProvider);
-            treeDataflowHelperFactories.add(indexDataflowHelperFactory);
-        }
-        ExternalDatasetIndexesCommitOperatorDescriptor op =
-                new ExternalDatasetIndexesCommitOperatorDescriptor(spec, treeDataflowHelperFactories);
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    public static JobSpecification buildAbortOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException {
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager();
-        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
-        AlgebricksPartitionConstraint constraints = null;
-        for (Index index : indexes) {
-            IFileSplitProvider indexSplitProvider;
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints =
-                        metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName());
-                indexSplitProvider = sAndConstraints.first;
-                constraints = sAndConstraints.second;
-            } else {
-                indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds,
-                        IndexingConstants.getFilesIndexName(ds.getDatasetName())).first;
-            }
-            IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storageMgr, indexSplitProvider);
-            treeDataflowHelperFactories.add(indexDataflowHelperFactory);
-        }
-        ExternalDatasetIndexesAbortOperatorDescriptor op =
-                new ExternalDatasetIndexesAbortOperatorDescriptor(spec, treeDataflowHelperFactories);
-
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-
-    }
-
-    public static JobSpecification buildRecoverOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException {
-        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager();
-        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
-        AlgebricksPartitionConstraint constraints = null;
-        for (Index index : indexes) {
-            IFileSplitProvider indexSplitProvider;
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints =
-                        metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName());
-                indexSplitProvider = sAndConstraints.first;
-                constraints = sAndConstraints.second;
-            } else {
-                indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds,
-                        IndexingConstants.getFilesIndexName(ds.getDatasetName())).first;
-            }
-            IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storageMgr, indexSplitProvider);
-            treeDataflowHelperFactories.add(indexDataflowHelperFactory);
-        }
-        ExternalDatasetIndexesRecoverOperatorDescriptor op =
-                new ExternalDatasetIndexesRecoverOperatorDescriptor(spec, treeDataflowHelperFactories);
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    public static boolean isFileIndex(Index index) {
-        return index.getIndexName().equals(IndexingConstants.getFilesIndexName(index.getDatasetName()));
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index aa37ebbe41..36a5351cca 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -27,7 +27,6 @@ import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -54,7 +53,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProv
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeLocalResourceFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterLocalResourceFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.storage.common.IResourceFactory;
@@ -130,8 +128,8 @@ public class RTreeResourceFactoryProvider implements IResourceFactoryProvider {
 
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            secondaryTypeTraits[numNestedSecondaryKeyFields + i] = (dataset.getDatasetType() == DatasetType.INTERNAL)
-                    ? primaryTypeTraits[i] : IndexingConstants.getTypeTraits(i);
+            secondaryTypeTraits[numNestedSecondaryKeyFields + i] =
+                    (dataset.getDatasetType() == DatasetType.INTERNAL) ? primaryTypeTraits[i] : null;
         }
         int[] rtreeFields = null;
         if (filterTypeTraits != null && filterTypeTraits.length > 0) {
@@ -154,10 +152,9 @@ public class RTreeResourceFactoryProvider implements IResourceFactoryProvider {
         IBinaryComparatorFactory[] rtreeCmpFactories = getCmpFactories(mdProvider, index, recordType, metaType);
         int[] secondaryFilterFields = (filterTypeTraits != null && filterTypeTraits.length > 0)
                 ? new int[] { numNestedSecondaryKeyFields + numPrimaryKeys } : null;
-        IBinaryComparatorFactory[] btreeCompFactories =
-                dataset.getDatasetType() == DatasetType.EXTERNAL ? IndexingConstants.getBuddyBtreeComparatorFactories()
-                        : getComparatorFactoriesForDeletedKeyBTree(secondaryTypeTraits, primaryComparatorFactories,
-                                secondaryComparatorFactories);
+        IBinaryComparatorFactory[] btreeCompFactories = dataset.getDatasetType() == DatasetType.EXTERNAL ? null
+                : getComparatorFactoriesForDeletedKeyBTree(secondaryTypeTraits, primaryComparatorFactories,
+                        secondaryComparatorFactories);
         ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             AsterixVirtualBufferCacheProvider vbcProvider =
@@ -169,13 +166,7 @@ public class RTreeResourceFactoryProvider implements IResourceFactoryProvider {
                     linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories,
                     typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
         } else {
-            return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
-                    filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
-                    pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
-                    mergePolicyProperties, true, btreeCompFactories, valueProviderFactories, rTreePolicyType,
-                    linearizeCmpFactory, rtreeFields, new int[] { numNestedSecondaryKeyFields }, isPointMBR,
-                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
-                    typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+            return null;
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 5a9753c1ec..01d1c383ed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -23,7 +23,6 @@ import java.util.List;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -48,9 +47,7 @@ import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
@@ -71,56 +68,6 @@ public class SecondaryBTreeOperationsHelper extends SecondaryTreeIndexOperations
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
         boolean excludeUnknown = excludeUnknownKeys(index, indexDetails, anySecondaryKeyIsNullable);
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            /*
-             * In case of external data,
-             * this method is used to build loading jobs for both initial load on index creation
-             * and transaction load on dataset referesh
-             */
-
-            // Create external indexing scan operator
-            ExternalScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
-
-            // Assign op.
-            AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                    createExternalAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
-
-            // If any of the secondary fields are nullable, then add a select op that filters nulls.
-            AlgebricksMetaOperatorDescriptor selectOp = null;
-            if (excludeUnknown) {
-                selectOp =
-                        createFilterAllUnknownsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
-            }
-            // Sort by secondary keys.
-            ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-            // Create secondary BTree bulk load op.
-            AbstractSingleActivityOperatorDescriptor secondaryBulkLoadOp;
-            IOperatorDescriptor root;
-            if (externalFiles != null) {
-                // Transaction load
-                secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, fieldPermutation, dataflowHelperFactory,
-                        StorageConstants.DEFAULT_TREE_FILL_FACTOR);
-            } else {
-                // Initial load
-                secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, dataflowHelperFactory,
-                        StorageConstants.DEFAULT_TREE_FILL_FACTOR);
-            }
-            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
-            sinkRuntimeFactory.setSourceLocation(sourceLoc);
-            AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
-            metaOp.setSourceLocation(sourceLoc);
-            spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
-            root = metaOp;
-            spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
-            if (excludeUnknown) {
-                spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-                spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
-            } else {
-                spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
-            }
-            spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-            spec.addRoot(root);
-            spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
             return spec;
         } else {
             // job spec:
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index c8fd5461c7..25b138dbd9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -26,15 +26,10 @@ import java.util.Set;
 import java.util.function.Supplier;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalIndexBulkLoadOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalIndexBulkModifyOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
@@ -44,7 +39,6 @@ import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.AbstractFunctionDescriptor;
@@ -84,7 +78,6 @@ import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
 @SuppressWarnings("rawtypes")
@@ -459,17 +452,6 @@ public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexO
         return treeIndexBulkLoadOp;
     }
 
-    protected TreeIndexBulkLoadOperatorDescriptor createExternalIndexBulkLoadOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
-        ExternalIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkLoadOperatorDescriptor(spec,
-                secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
-                ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider), null);
-        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
-                secondaryPartitionConstraint);
-        return treeIndexBulkLoadOp;
-    }
-
     public AlgebricksMetaOperatorDescriptor createFilterAllUnknownsSelectOp(JobSpecification spec,
             int numSecondaryKeyFields, RecordDescriptor secondaryRecDesc) throws AlgebricksException {
         return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, OrDescriptor::new);
@@ -516,39 +498,6 @@ public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexO
         return asterixSelectOp;
     }
 
-    // This method creates a source indexing operator for external data
-    protected ExternalScanOperatorDescriptor createExternalIndexingOp(JobSpecification spec)
-            throws AlgebricksException {
-        // A record + primary keys
-        ISerializerDeserializer[] serdes = new ISerializerDeserializer[1 + numPrimaryKeys];
-        ITypeTraits[] typeTraits = new ITypeTraits[1 + numPrimaryKeys];
-        // payload serde and type traits for the record slot
-        serdes[0] = payloadSerde;
-        typeTraits[0] = TypeTraitProvider.INSTANCE.getTypeTrait(itemType);
-        //  serdes and type traits for rid fields
-        for (int i = 1; i < serdes.length; i++) {
-            serdes[i] = IndexingConstants.getSerializerDeserializer(i - 1);
-            typeTraits[i] = IndexingConstants.getTypeTraits(i - 1);
-        }
-        // output record desc
-        RecordDescriptor indexerDesc = new RecordDescriptor(serdes, typeTraits);
-
-        // Create the operator and its partition constraits
-        Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
-        try {
-            indexingOpAndConstraints = ExternalIndexingOperations.createExternalIndexingOp(spec, metadataProvider,
-                    dataset, itemType, indexerDesc, externalFiles, sourceLoc);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
-                indexingOpAndConstraints.second);
-
-        // Set the primary partition constraints to this partition constraints
-        primaryPartitionConstraint = indexingOpAndConstraints.second;
-        return indexingOpAndConstraints.first;
-    }
-
     protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec, int numSecondaryKeys,
             RecordDescriptor secondaryRecDesc) {
         int[] outColumns = new int[numSecondaryKeys];
@@ -570,33 +519,6 @@ public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexO
                 new RecordDescriptor[] { secondaryRecDesc });
     }
 
-    protected ExternalIndexBulkModifyOperatorDescriptor createExternalIndexBulkModifyOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
-        // create a list of file ids
-        int numOfDeletedFiles = 0;
-        for (ExternalFile file : externalFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
-                numOfDeletedFiles++;
-            }
-        }
-        int[] deletedFiles = new int[numOfDeletedFiles];
-        int i = 0;
-        for (ExternalFile file : externalFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
-                deletedFiles[i] = file.getFileNumber();
-            }
-        }
-        ExternalIndexBulkModifyOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkModifyOperatorDescriptor(
-                spec, dataflowHelperFactory, deletedFiles, fieldPermutation, fillFactor, false, numElementsHint, null);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
-                secondaryPartitionConstraint);
-        return treeIndexBulkLoadOp;
-    }
-
-    public List<ExternalFile> getExternalFiles() {
-        return externalFiles;
-    }
-
     public void setExternalFiles(List<ExternalFile> externalFiles) {
         this.externalFiles = externalFiles;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index e36e71fab9..792d495924 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -24,7 +24,6 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -50,8 +49,6 @@ import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -254,63 +251,6 @@ public class SecondaryRTreeOperationsHelper extends SecondaryTreeIndexOperations
             spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         } else {
             // External dataset
-            /*
-             * In case of external data, this method is used to build loading jobs for both
-             * initial load on index creation
-             * and transaction load on dataset referesh
-             */
-            // Create external indexing scan operator
-            ExternalScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
-            AbstractOperatorDescriptor sourceOp = primaryScanOp;
-            if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
-                sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
-                spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
-            }
-            // Assign op.
-            AlgebricksMetaOperatorDescriptor asterixAssignOp = createExternalAssignOp(spec,
-                    numNestedSecondaryKeFieldsConsideringPointMBR, secondaryRecDescConsideringPointMBR);
-
-            // If any of the secondary fields are nullable, then add a select op that filters nulls.
-            AlgebricksMetaOperatorDescriptor selectOp = null;
-            if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
-                selectOp = createFilterAnyUnknownSelectOp(spec, numNestedSecondaryKeFieldsConsideringPointMBR,
-                        secondaryRecDescConsideringPointMBR);
-            }
-
-            // Sort by secondary keys.
-            ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
-                    new IBinaryComparatorFactory[] {
-                            MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length) },
-                    isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc);
-            // Create secondary RTree bulk load op.
-            IOperatorDescriptor root;
-            AbstractSingleActivityOperatorDescriptor secondaryBulkLoadOp;
-            if (externalFiles != null) {
-                // Transaction load
-                secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, fieldPermutation,
-                        indexDataflowHelperFactory, StorageConstants.DEFAULT_TREE_FILL_FACTOR);
-            } else {
-                // Initial load
-                secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, indexDataflowHelperFactory,
-                        StorageConstants.DEFAULT_TREE_FILL_FACTOR);
-            }
-            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
-            sinkRuntimeFactory.setSourceLocation(sourceLoc);
-            AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
-            metaOp.setSourceLocation(sourceLoc);
-            spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
-            root = metaOp;
-            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
-            if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
-                spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-                spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
-            } else {
-                spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
-            }
-            spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-            spec.addRoot(root);
-            spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         }
         return spec;
     }
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.1.script.sqlpp b/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.1.script.sqlpp
deleted file mode 100644
index db1a058e43..0000000000
--- a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.1.script.sqlpp
+++ /dev/null
@@ -1,19 +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.
-#
-create_and_start.sh
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.2.ddl.sqlpp b/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.2.ddl.sqlpp
deleted file mode 100644
index 3eddfc56eb..0000000000
--- a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.2.ddl.sqlpp
+++ /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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-drop dataverse test if exists;
-create dataverse test;
-
-use test;
-
-create type EmployeeType as closed {
- id: int64,
- name: string,
- age: int64
-};
-
-create external dataset EmployeeDataset(EmployeeType)
-using hdfs
-(("hdfs"="hdfs://127.0.0.1:31888"),
-("path"="/asterix/external-indexing-test.txt"),
-("input-format"="text-input-format"),
-("format"="delimited-text"),
-("delimiter"="|"));
-
-create index EmployeeAgeIdx on EmployeeDataset(age);
-
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.3.script.sqlpp b/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.3.script.sqlpp
deleted file mode 100644
index d844d13284..0000000000
--- a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.3.script.sqlpp
+++ /dev/null
@@ -1,19 +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.
-#
-stop_and_start.sh
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.4.query.sqlpp b/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.4.query.sqlpp
deleted file mode 100644
index e53d2ce98f..0000000000
--- a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.4.query.sqlpp
+++ /dev/null
@@ -1,31 +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.
- */
-/*
-* Description  : Create an external dataset that contains records stored with text hdfs file format.
-                 Build an index over the external dataset age attribute
-                 Perform a query over the dataset using the index.
-* Expected Res : Success
-* Date         : 3rd Jan 2014
-*/
-use test;
-
-SELECT VALUE e
-FROM EmployeeDataset e
-WHERE e.age = 22
-ORDER BY e.id;
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.5.script.sqlpp b/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.5.script.sqlpp
deleted file mode 100644
index f43dec7c88..0000000000
--- a/asterixdb/asterix-server/src/test/resources/transactionts/queries/query_after_restart/external_index/external_index.5.script.sqlpp
+++ /dev/null
@@ -1,19 +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.
-#
-stop_and_delete.sh
diff --git a/asterixdb/asterix-server/src/test/resources/transactionts/testsuite.xml b/asterixdb/asterix-server/src/test/resources/transactionts/testsuite.xml
index 0aab20dc37..7f3cfaedf8 100644
--- a/asterixdb/asterix-server/src/test/resources/transactionts/testsuite.xml
+++ b/asterixdb/asterix-server/src/test/resources/transactionts/testsuite.xml
@@ -23,11 +23,6 @@
         <output-dir compare="Text">dataset-with-meta-record</output-dir>
       </compilation-unit>
     </test-case>
-    <test-case FilePath="query_after_restart">
-      <compilation-unit name="external_index">
-        <output-dir compare="Text">external_index</output-dir>
-      </compilation-unit>
-    </test-case>
     <test-case FilePath="query_after_restart">
       <compilation-unit name="big_object_20M">
         <output-dir compare="Text">big_object_20M</output-dir>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
deleted file mode 100644
index ee4c4f7c34..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ /dev/null
@@ -1,104 +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.hyracks.storage.am.lsm.btree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.io.IJsonSerializable;
-import org.apache.hyracks.api.io.IPersistedResourceRegistry;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ExternalBTreeLocalResource extends LSMBTreeLocalResource {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
-            IStorageManager storageManager, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
-            boolean isSecondaryNoIncrementalMaintenance) {
-        super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
-                storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
-                btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
-                metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
-                isSecondaryNoIncrementalMaintenance);
-    }
-
-    private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
-            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
-        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
-    }
-
-    @Override
-    public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
-        IIOManager ioManager = serviceCtx.getIoManager();
-        FileReference file = ioManager.resolve(path);
-        ioOpCallbackFactory.initialize(serviceCtx, this);
-        pageWriteCallbackFactory.initialize(serviceCtx, this);
-        return LSMBTreeUtil.createExternalBTree(ioManager, file, storageManager.getBufferCache(serviceCtx), typeTraits,
-                cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
-                opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
-                ioOpCallbackFactory, pageWriteCallbackFactory, durable, metadataPageManagerFactory,
-                serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
-    }
-
-    @Override
-    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
-        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
-        super.appendToJson(jsonObject, registry);
-        return jsonObject;
-    }
-
-    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
-            throws HyracksDataException {
-        LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
-        return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
-                lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
deleted file mode 100644
index 3a2805756e..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ /dev/null
@@ -1,67 +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.hyracks.storage.am.lsm.btree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
-
-public class ExternalBTreeLocalResourceFactory extends LSMBTreeLocalResourceFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) {
-        super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
-                opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
-                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
-    }
-
-    @Override
-    public LsmResource createResource(FileReference fileRef) {
-        return new ExternalBTreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
-                bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
-                mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
-                opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
-                isSecondaryNoIncrementalMaintenance);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
deleted file mode 100644
index f2c526324d..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.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.hyracks.storage.am.lsm.btree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.io.IJsonSerializable;
-import org.apache.hyracks.api.io.IPersistedResourceRegistry;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-/**
- * The local resource for disk only lsm btree with buddy tree
- */
-public class ExternalBTreeWithBuddyLocalResource extends LSMBTreeLocalResource {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalBTreeWithBuddyLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] buddyBtreeFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
-            IStorageManager storageManager, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
-            boolean isSecondaryNoIncrementalMaintenance) {
-        super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
-                mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
-                filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
-                metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
-                isSecondaryNoIncrementalMaintenance);
-    }
-
-    private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
-            int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
-            boolean hasBloomFilter, boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
-        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
-    }
-
-    @Override
-    public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
-        IIOManager ioManager = serviceCtx.getIoManager();
-        FileReference file = ioManager.resolve(path);
-        ioOpCallbackFactory.initialize(serviceCtx, this);
-        pageWriteCallbackFactory.initialize(serviceCtx, this);
-        return LSMBTreeUtil.createExternalBTreeWithBuddy(ioManager, file, storageManager.getBufferCache(serviceCtx),
-                typeTraits, cmpFactories, bloomFilterFalsePositiveRate,
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
-                opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
-                ioOpCallbackFactory, pageWriteCallbackFactory, bloomFilterKeyFields, durable,
-                metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
-    }
-
-    @Override
-    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
-        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
-        super.appendToJson(jsonObject, registry);
-        return jsonObject;
-    }
-
-    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
-            throws HyracksDataException {
-        LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
-        return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
-                lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
deleted file mode 100644
index 995a23bd97..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ /dev/null
@@ -1,67 +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.hyracks.storage.am.lsm.btree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
-
-public class ExternalBTreeWithBuddyLocalResourceFactory extends LSMBTreeLocalResourceFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalBTreeWithBuddyLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
-            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) {
-        super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
-                opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
-                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
-    }
-
-    @Override
-    public LsmResource createResource(FileReference fileRef) {
-        return new ExternalBTreeWithBuddyLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
-                bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
-                mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
-                opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
-                isSecondaryNoIncrementalMaintenance);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
deleted file mode 100644
index e727dc1c56..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ /dev/null
@@ -1,604 +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.hyracks.storage.am.lsm.btree.impls;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
-import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.IIndexCursorStats;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.IndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.util.trace.ITracer;
-
-/**
- * This is an lsm b-tree that does not have memory component and is modified
- * only by bulk loading and addition of disk components as of this point, it is
- * intended for use with external dataset indexes only.
- *
- * @author alamouda
- */
-public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
-
-    private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
-    // This component factory has to be different since it uses different tuple
-    // writer in it's leaf frames to support inserting both
-    // regular and delete tuples
-    private final ILSMDiskComponentFactory transactionComponentFactory;
-    // A second disk component list that will be used when a transaction is
-    // committed and will be seen by subsequent accessors
-    private final List<ILSMDiskComponent> secondDiskComponents;
-    // A pointer that points to the current most recent list (either
-    // diskComponents = 0, or secondDiskComponents = 1). It starts with -1 to
-    // indicate first time activation
-    private int version = 0;
-
-    private final ITreeIndexFrameFactory interiorFrameFactory;
-
-    //TODO remove BloomFilter from external dataset's secondary LSMBTree index
-    public ExternalBTree(IIOManager ioManager, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
-            IBufferCache bufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
-            ILSMDiskComponentFactory bulkLoadComponentFactory, ILSMDiskComponentFactory transactionComponentFactory,
-            double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean durable, ITracer tracer) throws HyracksDataException {
-        super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, bufferCache, fileManager, componentFactory,
-                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, false, durable, tracer);
-        this.transactionComponentFactory = transactionComponentFactory;
-        this.secondDiskComponents = new LinkedList<>();
-        this.interiorFrameFactory = interiorFrameFactory;
-    }
-
-    @Override
-    public ExternalIndexHarness getHarness() {
-        return (ExternalIndexHarness) super.getHarness();
-    }
-
-    // The subsume merged components is overridden to account for:
-    // Maintaining two versions of the index
-    @Override
-    public void subsumeMergedComponents(ILSMDiskComponent newComponent, List<ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-        }
-
-        // Check if merge will affect the older list
-        if (olderList.containsAll(mergedComponents)) {
-            int swapIndex = olderList.indexOf(mergedComponents.get(0));
-            olderList.removeAll(mergedComponents);
-            olderList.add(swapIndex, newComponent);
-        }
-
-        // The new list will always have all the merged components
-        int swapIndex = newerList.indexOf(mergedComponents.get(0));
-        newerList.removeAll(mergedComponents);
-        newerList.add(swapIndex, newComponent);
-    }
-
-    // This method is used by the merge policy when it needs to check if a merge
-    // is needed.
-    // It only needs to return the newer list
-    @Override
-    public List<ILSMDiskComponent> getDiskComponents() {
-        if (version == 0) {
-            return diskComponents;
-        } else if (version == 1) {
-            return secondDiskComponents;
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-    // The only reason to override the following method is that it uses a different context object
-    @Override
-    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
-            throws HyracksDataException {
-        ExternalBTreeOpContext ctx = (ExternalBTreeOpContext) ictx;
-        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
-        ctx.getSearchInitialState().reset(pred, operationalComponents);
-        cursor.open(ctx.getSearchInitialState(), pred);
-    }
-
-    // This method creates the appropriate opContext for the targeted version
-    public ExternalBTreeOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
-        return new ExternalBTreeOpContext(this, insertLeafFrameFactory, deleteLeafFrameFactory, searchCallback,
-                ((LSMBTreeWithBloomFilterDiskComponentFactory) componentFactory).getBloomFilterKeyFields().length,
-                cmpFactories, targetVersion, getHarness(), tracer);
-    }
-
-    // The only reason to override the following method is that it uses a different context object
-    // in addition, determining whether or not to keep deleted tuples is different here
-    @Override
-    public LSMBTreeMergeOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
-        ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
-        opCtx.setOperation(IndexOperation.MERGE);
-        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
-        boolean returnDeletedTuples = false;
-        if (version == 0) {
-            if (ctx.getComponentHolder().get(ctx.getComponentHolder().size() - 1) != diskComponents
-                    .get(diskComponents.size() - 1)) {
-                returnDeletedTuples = true;
-            }
-        } else {
-            if (ctx.getComponentHolder().get(ctx.getComponentHolder().size() - 1) != secondDiskComponents
-                    .get(secondDiskComponents.size() - 1)) {
-                returnDeletedTuples = true;
-            }
-        }
-        IIndexCursorStats stats = new IndexCursorStats();
-        LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
-        BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(0)).getIndex();
-        BTree firstBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getIndex();
-        FileReference firstFile = firstBTree.getFileReference();
-        FileReference lastFile = lastBTree.getFileReference();
-        LSMComponentFileReferences relMergeFileRefs =
-                fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
-        ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-        LSMBTreeMergeOperation mergeOp = new LSMBTreeMergeOperation(accessor, cursor, stats,
-                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
-        ioOpCallback.scheduled(mergeOp);
-        return mergeOp;
-    }
-
-    // This function should only be used when a transaction fail. it doesn't
-    // take any parameters since there can only be
-    // a single transaction and hence a single transaction component on disk
-    public void deleteTransactionComponent() throws HyracksDataException {
-        fileManager.deleteTransactionFiles();
-    }
-
-    // This function in an instance of this index is only used after a bulk load
-    // is successful
-    @Override
-    public void addDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        if (version == 0) {
-            diskComponents.add(0, c);
-        } else if (version == 1) {
-            secondDiskComponents.add(0, c);
-        }
-    }
-
-    // This function is used when a new component is to be committed.
-    @Override
-    public void commitTransactionDiskComponent(ILSMDiskComponent newComponent) throws HyracksDataException {
-        // determine which list is the new one and flip the pointer
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-            version = 1;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-            version = 0;
-        }
-        // Remove components from list
-        olderList.clear();
-        // Add components
-        olderList.addAll(newerList);
-        if (newComponent != null) {
-            // Add it to the list
-            olderList.add(0, newComponent);
-        }
-    }
-
-    @Override
-    public synchronized void activate() throws HyracksDataException {
-        if (isActive) {
-            throw new HyracksDataException("Failed to activate the index since it is already activated.");
-        }
-        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
-            //First time activation
-            List<LSMComponentFileReferences> validFileReferences;
-            validFileReferences = fileManager.cleanupAndGetValidFiles();
-            for (LSMComponentFileReferences lsmComponentFileReferences : validFileReferences) {
-                ILSMDiskComponent component =
-                        createDiskComponent(componentFactory, lsmComponentFileReferences.getInsertIndexFileReference(),
-                                null, lsmComponentFileReferences.getBloomFilterFileReference(), false);
-                diskComponents.add(component);
-                secondDiskComponents.add(component);
-            }
-            getHarness().indexFirstTimeActivated();
-        } else {
-            // This index has been opened before
-            for (ILSMDiskComponent c : diskComponents) {
-                c.activate(false);
-            }
-            for (ILSMDiskComponent c : secondDiskComponents) {
-                // Only activate non shared components
-                if (!diskComponents.contains(c)) {
-                    c.activate(false);
-                }
-            }
-        }
-        isActive = true;
-    }
-
-    @Override
-    public synchronized void create() throws HyracksDataException {
-        super.create();
-        secondDiskComponents.clear();
-    }
-
-    @Override
-    public void deactivate(boolean flushOnExit) throws HyracksDataException {
-        if (!isActive) {
-            throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
-        }
-        for (ILSMDiskComponent c : diskComponents) {
-            c.deactivateAndPurge();
-        }
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            // Only deactivate non shared components
-            if (!diskComponents.contains(c)) {
-                c.deactivateAndPurge();
-            }
-        }
-        isActive = false;
-    }
-
-    // The clear method is not used anywhere in AsterixDB! we override it anyway
-    // to exit components first and deal with the two lists
-    @Override
-    public void clear() throws HyracksDataException {
-        if (!isActive) {
-            throw new HyracksDataException("Failed to clear the index since it is not activated.");
-        }
-        getHarness().indexClear();
-
-        for (ILSMDiskComponent c : diskComponents) {
-            c.deactivateAndDestroy();
-            // Remove from second list to avoid destroying twice
-            secondDiskComponents.remove(c);
-        }
-
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            c.deactivateAndDestroy();
-        }
-        diskComponents.clear();
-        secondDiskComponents.clear();
-        version = 0;
-    }
-
-    @Override
-    public void destroy() throws HyracksDataException {
-        if (isActive) {
-            throw new HyracksDataException("Failed to destroy the index since it is activated.");
-        }
-        for (ILSMDiskComponent c : diskComponents) {
-            c.destroy();
-            // Remove from second list to avoid destroying twice
-            secondDiskComponents.remove(c);
-        }
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            c.destroy();
-        }
-        diskComponents.clear();
-        secondDiskComponents.clear();
-        fileManager.deleteDirs();
-        version = 0;
-    }
-
-    // Not supported
-    @Override
-    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
-        throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree");
-    }
-
-    // Not supported
-    @Override
-    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
-        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
-    }
-
-    // Not supported
-    @Override
-    public ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException {
-        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
-    }
-
-    // Only support search and merge operations
-    @Override
-    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
-        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
-        List<ILSMDiskComponent> immutableComponents;
-        // Identify current list in case of a merge
-        if (version == 0) {
-            immutableComponents = diskComponents;
-        } else {
-            immutableComponents = secondDiskComponents;
-        }
-        ExternalBTreeOpContext opCtx = (ExternalBTreeOpContext) ctx;
-        operationalComponents.clear();
-        switch (ctx.getOperation()) {
-            case SEARCH:
-                if (opCtx.getTargetIndexVersion() == 0) {
-                    operationalComponents.addAll(diskComponents);
-                } else {
-                    operationalComponents.addAll(secondDiskComponents);
-                }
-                break;
-            case MERGE:
-                operationalComponents.addAll(ctx.getComponentsToBeMerged());
-                break;
-            case FULL_MERGE:
-                operationalComponents.addAll(immutableComponents);
-                break;
-            case REPLICATE:
-                operationalComponents.addAll(ctx.getComponentsToBeReplicated());
-                break;
-            default:
-                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
-        }
-    }
-
-    // For initial load
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
-            Map<String, Object> parameters) throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false, parameters);
-    }
-
-    // For transaction bulk load <- could consolidate with the above method ->
-    @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
-            Map<String, Object> parameters) throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
-    }
-
-    // The bulk loader used for both initial loading and transaction
-    // modifications
-    public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
-        private final ILSMDiskComponent component;
-        private final ILSMDiskComponentBulkLoader componentBulkLoader;
-        private final LoadOperation loadOp;
-
-        private final boolean isTransaction;
-
-        public LSMTwoPCBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
-            this.isTransaction = isTransaction;
-            // Create the appropriate target
-            LSMComponentFileReferences componentFileRefs;
-            if (isTransaction) {
-                try {
-                    componentFileRefs = fileManager.getNewTransactionFileReference();
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
-                }
-                component = createDiskComponent(transactionComponentFactory,
-                        componentFileRefs.getInsertIndexFileReference(), null,
-                        componentFileRefs.getBloomFilterFileReference(), true);
-            } else {
-                componentFileRefs = fileManager.getRelFlushFileReference();
-                component =
-                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                                componentFileRefs.getDeleteIndexFileReference(),
-                                componentFileRefs.getBloomFilterFileReference(), true);
-            }
-            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
-            loadOp.setNewComponent(component);
-            ioOpCallback.scheduled(loadOp);
-            ioOpCallback.beforeOperation(loadOp);
-            IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
-            componentBulkLoader = component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false,
-                    true, true, pageWriteCallback);
-        }
-
-        // It is expected that the mode was set to insert operation before
-        // calling add
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.add(tuple);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            try {
-                ioOpCallback.afterOperation(loadOp);
-                componentBulkLoader.end();
-                if (component.getComponentSize() > 0) {
-                    if (isTransaction) {
-                        // Since this is a transaction component, validate and
-                        // deactivate. it could later be added or deleted
-                        try {
-                            component.markAsValid(durable, loadOp);
-                        } finally {
-                            ioOpCallback.afterFinalize(loadOp);
-                        }
-                        component.deactivate();
-                    } else {
-                        ioOpCallback.afterFinalize(loadOp);
-                        getHarness().addBulkLoadedComponent(loadOp);
-                    }
-                }
-            } finally {
-                ioOpCallback.completed(loadOp);
-            }
-        }
-
-        // It is expected that the mode was set to delete operation before
-        // calling delete
-        @Override
-        public void delete(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.delete(tuple);
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            try {
-                componentBulkLoader.abort();
-            } finally {
-                ioOpCallback.completed(loadOp);
-            }
-        }
-
-        @Override
-        public void writeFailed(ICachedPage page, Throwable failure) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public boolean hasFailed() {
-            return componentBulkLoader.hasFailed();
-        }
-
-        @Override
-        public Throwable getFailure() {
-            return componentBulkLoader.getFailure();
-        }
-
-        @Override
-        public void force() throws HyracksDataException {
-            componentBulkLoader.force();
-        }
-    }
-
-    // The accessor for disk only indexes don't use modification callback and always carry the target index version with them
-    @Override
-    public ILSMIndexAccessor createAccessor(IIndexAccessParameters iap) {
-        ExternalBTreeOpContext opCtx = createOpContext(iap.getSearchOperationCallback(), version);
-        return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-    }
-
-    @Override
-    public ILSMIndexAccessor createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
-            throws HyracksDataException {
-        ExternalBTreeOpContext opCtx = createOpContext(searchCallback, targetIndexVersion);
-        return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-    }
-
-    @Override
-    public ITreeIndexFrameFactory getInteriorFrameFactory() {
-        return interiorFrameFactory;
-    }
-
-    @Override
-    public int getFieldCount() {
-        return cmpFactories.length;
-    }
-
-    @Override
-    public int getFileId() {
-        return -1;
-    }
-
-    @Override
-    public IMetadataPageManager getPageManager() {
-        return null;
-    }
-
-    @Override
-    public ITreeIndexFrameFactory getLeafFrameFactory() {
-        return insertLeafFrameFactory;
-    }
-
-    @Override
-    public int getCurrentVersion() {
-        return version;
-    }
-
-    @Override
-    public void setCurrentVersion(int version) {
-        this.version = version;
-    }
-
-    @Override
-    public List<ILSMDiskComponent> getFirstComponentList() {
-        return diskComponents;
-    }
-
-    @Override
-    public List<ILSMDiskComponent> getSecondComponentList() {
-        return secondDiskComponents;
-    }
-
-    @Override
-    public void commitTransaction() throws HyracksDataException {
-        LSMComponentFileReferences componentFileReferences = fileManager.getTransactionFileReferenceForCommit();
-        ILSMDiskComponent component = null;
-        if (componentFileReferences != null) {
-            component = createDiskComponent(componentFactory, componentFileReferences.getInsertIndexFileReference(),
-                    null, componentFileReferences.getBloomFilterFileReference(), false);
-        }
-        getHarness().addTransactionComponents(component);
-    }
-
-    @Override
-    public void abortTransaction() throws HyracksDataException {
-        fileManager.deleteTransactionFiles();
-    }
-
-    @Override
-    public void recoverTransaction() throws HyracksDataException {
-        fileManager.recoverTransaction();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
deleted file mode 100644
index 0b49b03a7d..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
+++ /dev/null
@@ -1,82 +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.hyracks.storage.am.lsm.btree.impls;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.MultiComparator;
-import org.apache.hyracks.util.trace.ITracer;
-
-public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext {
-    private IBTreeLeafFrame insertLeafFrame;
-    private IBTreeLeafFrame deleteLeafFrame;
-    private final MultiComparator cmp;
-    private final MultiComparator bloomFilterCmp;
-    private final int targetIndexVersion;
-    private LSMBTreeCursorInitialState searchInitialState;
-
-    public ExternalBTreeOpContext(ILSMIndex index, ITreeIndexFrameFactory insertLeafFrameFactory,
-            ITreeIndexFrameFactory deleteLeafFrameFactory, ISearchOperationCallback searchCallback,
-            int numBloomFilterKeyFields, IBinaryComparatorFactory[] cmpFactories, int targetIndexVersion,
-            ILSMHarness lsmHarness, ITracer tracer) {
-        super(index, null, null, null, searchCallback, null, tracer);
-        if (cmpFactories != null) {
-            this.cmp = MultiComparator.create(cmpFactories);
-        } else {
-            this.cmp = null;
-        }
-        bloomFilterCmp = MultiComparator.create(cmpFactories, 0, numBloomFilterKeyFields);
-        this.insertLeafFrame = (IBTreeLeafFrame) insertLeafFrameFactory.createFrame();
-        this.deleteLeafFrame = (IBTreeLeafFrame) deleteLeafFrameFactory.createFrame();
-        if (insertLeafFrame != null && this.cmp != null) {
-            insertLeafFrame.setMultiComparator(cmp);
-        }
-        if (deleteLeafFrame != null && this.cmp != null) {
-            deleteLeafFrame.setMultiComparator(cmp);
-        }
-        this.targetIndexVersion = targetIndexVersion;
-        searchInitialState = new LSMBTreeCursorInitialState(insertLeafFrameFactory, cmp, bloomFilterCmp, lsmHarness,
-                null, searchCallback, null);
-    }
-
-    @Override
-    public void setCurrentMutableComponentId(int currentMutableComponentId) {
-        // Do nothing: this method should never be called for this class
-    }
-
-    // Used by indexes with global transaction
-    public int getTargetIndexVersion() {
-        return targetIndexVersion;
-    }
-
-    public LSMBTreeCursorInitialState getSearchInitialState() {
-        return searchInitialState;
-    }
-
-    @Override
-    public void destroy() throws HyracksDataException {
-        // No Op
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
deleted file mode 100644
index 9855571eb3..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ /dev/null
@@ -1,698 +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.hyracks.storage.am.lsm.btree.impls;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
-import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
-import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.IIndexCursorStats;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.IndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.util.trace.ITracer;
-
-public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeIndex, ITwoPCIndex {
-
-    private final IBinaryComparatorFactory[] btreeCmpFactories;
-    private final IBinaryComparatorFactory[] buddyBtreeCmpFactories;
-    private final int[] buddyBTreeFields;
-
-    // Common for in-memory and on-disk components.
-    private final ITreeIndexFrameFactory btreeInteriorFrameFactory;
-    private final ITreeIndexFrameFactory btreeLeafFrameFactory;
-    private final ITreeIndexFrameFactory buddyBtreeLeafFrameFactory;
-
-    // A second disk component list that will be used when a transaction is
-    // committed and will be seen by subsequent accessors
-    private final List<ILSMDiskComponent> secondDiskComponents;
-    private int version = 0;
-
-    public ExternalBTreeWithBuddy(IIOManager ioManager, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, ITreeIndexFrameFactory buddyBtreeLeafFrameFactory,
-            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
-            ILSMDiskComponentFactory bulkLoadComponentFactory, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IBinaryComparatorFactory[] btreeCmpFactories, IBinaryComparatorFactory[] buddyBtreeCmpFactories,
-            int[] buddyBTreeFields, boolean durable, ITracer tracer) throws HyracksDataException {
-        super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, bulkLoadComponentFactory,
-                durable, tracer);
-        this.btreeCmpFactories = btreeCmpFactories;
-        this.buddyBtreeCmpFactories = buddyBtreeCmpFactories;
-        this.buddyBTreeFields = buddyBTreeFields;
-        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
-        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
-        this.buddyBtreeLeafFrameFactory = buddyBtreeLeafFrameFactory;
-        this.secondDiskComponents = new LinkedList<>();
-    }
-
-    @Override
-    public void create() throws HyracksDataException {
-        super.create();
-        secondDiskComponents.clear();
-    }
-
-    @Override
-    public void activate() throws HyracksDataException {
-        if (isActive) {
-            throw new HyracksDataException("Failed to activate the index since it is already activated.");
-        }
-        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
-            //First time activation
-            List<LSMComponentFileReferences> validFileReferences;
-            validFileReferences = fileManager.cleanupAndGetValidFiles();
-            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-                ILSMDiskComponent component;
-                component =
-                        createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(),
-                                lsmComonentFileReference.getDeleteIndexFileReference(),
-                                lsmComonentFileReference.getBloomFilterFileReference(), false);
-                diskComponents.add(component);
-                secondDiskComponents.add(component);
-            }
-            ((ExternalIndexHarness) getHarness()).indexFirstTimeActivated();
-        } else {
-            // This index has been opened before or is brand new with no
-            // components. It should also maintain the version pointer
-            for (ILSMDiskComponent c : diskComponents) {
-                c.activate(false);
-            }
-            for (ILSMDiskComponent c : secondDiskComponents) {
-                // Only activate non shared components
-                if (!diskComponents.contains(c)) {
-                    c.activate(false);
-                }
-            }
-        }
-        isActive = true;
-    }
-
-    @Override
-    public void clear() throws HyracksDataException {
-        if (!isActive) {
-            throw new HyracksDataException("Failed to clear the index since it is not activated.");
-        }
-        ((ExternalIndexHarness) getHarness()).indexClear();
-        for (ILSMDiskComponent c : diskComponents) {
-            c.deactivateAndDestroy();
-            // Remove from second list to avoid destroying twice
-            secondDiskComponents.remove(c);
-        }
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            c.deactivateAndDestroy();
-        }
-        diskComponents.clear();
-        secondDiskComponents.clear();
-        version = 0;
-    }
-
-    @Override
-    public void destroy() throws HyracksDataException {
-        if (isActive) {
-            throw new HyracksDataException("Failed to destroy the index since it is activated.");
-        }
-        for (ILSMDiskComponent c : diskComponents) {
-            c.destroy();
-            // Remove from second list to avoid destroying twice
-            secondDiskComponents.remove(c);
-        }
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            c.destroy();
-        }
-        diskComponents.clear();
-        secondDiskComponents.clear();
-        fileManager.deleteDirs();
-        version = 0;
-    }
-
-    @Override
-    public ILSMIndexAccessor createAccessor(IIndexAccessParameters iap) throws HyracksDataException {
-        return new LSMTreeIndexAccessor(getHarness(), createOpContext(iap.getSearchOperationCallback(), version),
-                ctx -> new LSMBTreeWithBuddySearchCursor(ctx, buddyBTreeFields));
-    }
-
-    // The subsume merged components is overridden to account for:
-    // Maintaining two versions of the index
-    @Override
-    public void subsumeMergedComponents(ILSMDiskComponent newComponent, List<ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-        }
-
-        // Check if merge will affect the older list
-        if (olderList.containsAll(mergedComponents)) {
-            int swapIndex = olderList.indexOf(mergedComponents.get(0));
-            olderList.removeAll(mergedComponents);
-            olderList.add(swapIndex, newComponent);
-        }
-
-        // The new list will always have all the merged components
-        int swapIndex = newerList.indexOf(mergedComponents.get(0));
-        newerList.removeAll(mergedComponents);
-        newerList.add(swapIndex, newComponent);
-    }
-
-    // For initial load
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
-            Map<String, Object> parameters) throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false, parameters);
-    }
-
-    // For transaction bulk load <- could consolidate with the above method ->
-    @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
-            Map<String, Object> parameters) throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
-    }
-
-    @Override
-    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
-        throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree");
-    }
-
-    @Override
-    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
-            throws HyracksDataException {
-        ExternalBTreeWithBuddyOpContext ctx = (ExternalBTreeWithBuddyOpContext) ictx;
-        List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
-        ctx.getSearchInitialState().setOperationalComponents(operationalComponents);
-        cursor.open(ctx.getSearchInitialState(), pred);
-    }
-
-    @Override
-    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
-        throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
-    }
-
-    @Override
-    public ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException {
-        throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
-    }
-
-    protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
-            throws HyracksDataException {
-        BTree lastTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getIndex();
-        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
-                .getIndex();
-        FileReference firstFile = firstTree.getFileReference();
-        FileReference lastFile = lastTree.getFileReference();
-        LSMComponentFileReferences fileRefs =
-                fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
-        return fileRefs;
-    }
-
-    @Override
-    public LSMBTreeWithBuddyMergeOperation createMergeOperation(ILSMIndexOperationContext ctx)
-            throws HyracksDataException {
-        ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
-        bctx.setOperation(IndexOperation.MERGE);
-        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
-        IIndexCursorStats stats = new IndexCursorStats();
-        LSMBTreeWithBuddySortedCursor cursor = new LSMBTreeWithBuddySortedCursor(bctx, buddyBTreeFields, stats);
-        LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
-        ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), bctx,
-                opCtx -> new LSMBTreeWithBuddySearchCursor(opCtx, buddyBTreeFields));
-
-        // Since we have two lists of components, to tell whether we need to
-        // keep deleted tuples, we need to know
-        // which list to check against and we need to synchronize for this
-        boolean keepDeleteTuples = false;
-        if (version == 0) {
-            keepDeleteTuples = mergingComponents.get(mergingComponents.size() - 1) != diskComponents
-                    .get(diskComponents.size() - 1);
-        } else {
-            keepDeleteTuples = mergingComponents.get(mergingComponents.size() - 1) != secondDiskComponents
-                    .get(secondDiskComponents.size() - 1);
-        }
-
-        LSMBTreeWithBuddyMergeOperation mergeOp = new LSMBTreeWithBuddyMergeOperation(accessor, cursor, stats,
-                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
-                relMergeFileRefs.getBloomFilterFileReference(), ioOpCallback,
-                fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples);
-        ioOpCallback.scheduled(mergeOp);
-        return mergeOp;
-
-    }
-
-    // This method creates the appropriate opContext for the targeted version
-    public ExternalBTreeWithBuddyOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
-        return new ExternalBTreeWithBuddyOpContext(this, btreeCmpFactories, buddyBtreeCmpFactories, searchCallback,
-                targetVersion, getHarness(), btreeInteriorFrameFactory, btreeLeafFrameFactory,
-                buddyBtreeLeafFrameFactory, tracer);
-    }
-
-    @Override
-    public ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException {
-        LSMBTreeWithBuddyMergeOperation mergeOp = (LSMBTreeWithBuddyMergeOperation) operation;
-        IIndexCursor cursor = mergeOp.getCursor();
-        ISearchPredicate btreeSearchPred = new RangePredicate(null, null, true, true, null, null);
-        ILSMIndexOperationContext opCtx = ((LSMBTreeWithBuddySortedCursor) cursor).getOpCtx();
-        search(opCtx, cursor, btreeSearchPred);
-
-        ILSMDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                mergeOp.getBuddyBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
-
-        ILSMDiskComponentBulkLoader componentBulkLoader;
-
-        // In case we must keep the deleted-keys BuddyBTrees, then they must be
-        // merged *before* merging the b-trees so that
-        // lsmHarness.endSearch() is called once when the b-trees have been
-        // merged.
-
-        if (mergeOp.isKeepDeletedTuples()) {
-            // Keep the deleted tuples since the oldest disk component is not
-            // included in the merge operation
-            LSMBuddyBTreeMergeCursor buddyBtreeCursor = new LSMBuddyBTreeMergeCursor(opCtx, mergeOp.getCursorStats());
-            search(opCtx, buddyBtreeCursor, btreeSearchPred);
-
-            long numElements = 0L;
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
-                        .getBloomFilter().getNumElements();
-            }
-            componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false,
-                    false, pageWriteCallbackFactory.createPageWriteCallback());
-            try {
-                while (buddyBtreeCursor.hasNext()) {
-                    buddyBtreeCursor.next();
-                    ITupleReference tuple = buddyBtreeCursor.getTuple();
-                    componentBulkLoader.delete(tuple);
-                }
-            } finally {
-                buddyBtreeCursor.close();
-            }
-        } else {
-            componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false,
-                    pageWriteCallbackFactory.createPageWriteCallback());
-        }
-
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference frameTuple = cursor.getTuple();
-                componentBulkLoader.add(frameTuple);
-            }
-        } finally {
-            cursor.close();
-        }
-        componentBulkLoader.end();
-        return mergedComponent;
-    }
-
-    @Override
-    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
-        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
-        List<ILSMDiskComponent> immutableComponents;
-        // Identify current list in case of a merge
-        if (version == 0) {
-            immutableComponents = diskComponents;
-        } else {
-            immutableComponents = secondDiskComponents;
-        }
-
-        ExternalBTreeWithBuddyOpContext opCtx = (ExternalBTreeWithBuddyOpContext) ctx;
-        operationalComponents.clear();
-        switch (ctx.getOperation()) {
-            case SEARCH:
-                if (opCtx.getTargetIndexVersion() == 0) {
-                    operationalComponents.addAll(diskComponents);
-                } else {
-                    operationalComponents.addAll(secondDiskComponents);
-                }
-                break;
-            case MERGE:
-                operationalComponents.addAll(ctx.getComponentsToBeMerged());
-                break;
-            case FULL_MERGE:
-                operationalComponents.addAll(immutableComponents);
-            case REPLICATE:
-                operationalComponents.addAll(ctx.getComponentsToBeReplicated());
-                break;
-            case FLUSH:
-                // Do nothing. this is left here even though the index never
-                // performs flushes because a flush is triggered by
-                // dataset lifecycle manager when closing an index. Having no
-                // components is a no operation
-                break;
-            default:
-                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
-        }
-    }
-
-    // This function is used when a new component is to be committed -- is
-    // called by the harness.
-    @Override
-    public void commitTransactionDiskComponent(ILSMDiskComponent newComponent) throws HyracksDataException {
-
-        // determine which list is the new one and flip the pointer
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-            version = 1;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-            version = 0;
-        }
-        // Remove components from list
-        olderList.clear();
-        // Add components
-        olderList.addAll(newerList);
-        if (newComponent != null) {
-            // Add it to the list
-            olderList.add(0, newComponent);
-        }
-    }
-
-    @Override
-    public void deactivate(boolean flushOnExit) throws HyracksDataException {
-        if (!isActive) {
-            throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
-        }
-        // Even though, we deactivate the index, we don't exit components or
-        // modify any of the lists to make sure they
-        // are there if the index was opened again
-        for (ILSMDiskComponent c : diskComponents) {
-            c.deactivateAndPurge();
-        }
-        for (ILSMDiskComponent c : secondDiskComponents) {
-            // Only deactivate non shared components
-            if (!diskComponents.contains(c)) {
-                c.deactivateAndPurge();
-            }
-        }
-        isActive = false;
-    }
-
-    @Override
-    public ITreeIndexFrameFactory getLeafFrameFactory() {
-        return btreeLeafFrameFactory;
-    }
-
-    @Override
-    public ITreeIndexFrameFactory getInteriorFrameFactory() {
-        return btreeInteriorFrameFactory;
-    }
-
-    @Override
-    public IMetadataPageManager getPageManager() {
-        // This method should never be called for disk only indexes
-        return null;
-    }
-
-    @Override
-    public int getFieldCount() {
-        return btreeCmpFactories.length;
-    }
-
-    @Override
-    public int getRootPageId() {
-        // This method should never be called for this index
-        return 0;
-    }
-
-    @Override
-    public int getFileId() {
-        // This method should never be called for this index
-        return 0;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getComparatorFactories() {
-        return btreeCmpFactories;
-    }
-
-    // even though the index doesn't support record level modification, the
-    // accessor will try to do it
-    // we could throw the exception here but we don't. it will eventually be
-    // thrown by the index itself
-
-    // The bulk loader used for both initial loading and transaction
-    // modifications
-    public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
-        private final ILSMDiskComponent component;
-        private final LoadOperation loadOp;
-        private final ILSMDiskComponentBulkLoader componentBulkLoader;
-        private final boolean isTransaction;
-
-        public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
-            this.isTransaction = isTransaction;
-            // Create the appropriate target
-            LSMComponentFileReferences componentFileRefs;
-            if (isTransaction) {
-                try {
-                    componentFileRefs = fileManager.getNewTransactionFileReference();
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
-                }
-                component =
-                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                                componentFileRefs.getDeleteIndexFileReference(),
-                                componentFileRefs.getBloomFilterFileReference(), true);
-            } else {
-                componentFileRefs = fileManager.getRelFlushFileReference();
-                component =
-                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                                componentFileRefs.getDeleteIndexFileReference(),
-                                componentFileRefs.getBloomFilterFileReference(), true);
-            }
-
-            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
-            loadOp.setNewComponent(component);
-            ioOpCallback.scheduled(loadOp);
-            ioOpCallback.beforeOperation(loadOp);
-            componentBulkLoader = component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false,
-                    true, false, pageWriteCallbackFactory.createPageWriteCallback());
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.add(tuple);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            try {
-                ioOpCallback.afterOperation(loadOp);
-                componentBulkLoader.end();
-                if (component.getComponentSize() > 0) {
-                    if (isTransaction) {
-                        // Since this is a transaction component, validate and
-                        // deactivate. it could later be added or deleted
-                        try {
-                            component.markAsValid(durable, loadOp);
-                        } finally {
-                            ioOpCallback.afterFinalize(loadOp);
-                        }
-                        component.deactivate();
-                    } else {
-                        ioOpCallback.afterFinalize(loadOp);
-                        getHarness().addBulkLoadedComponent(loadOp);
-                    }
-                }
-            } finally {
-                ioOpCallback.completed(loadOp);
-            }
-        }
-
-        @Override
-        public void delete(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.delete(tuple);
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            try {
-                try {
-                    componentBulkLoader.abort();
-                } finally {
-                    ioOpCallback.afterFinalize(loadOp);
-                }
-            } finally {
-                ioOpCallback.completed(loadOp);
-            }
-        }
-
-        @Override
-        public void writeFailed(ICachedPage page, Throwable failure) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public boolean hasFailed() {
-            return componentBulkLoader.hasFailed();
-        }
-
-        @Override
-        public Throwable getFailure() {
-            return componentBulkLoader.getFailure();
-        }
-
-        @Override
-        public void force() throws HyracksDataException {
-            componentBulkLoader.force();
-        }
-    }
-
-    @Override
-    public ILSMIndexAccessor createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
-            throws HyracksDataException {
-        return new LSMTreeIndexAccessor(getHarness(), createOpContext(searchCallback, targetIndexVersion),
-                ctx -> new LSMBTreeWithBuddySearchCursor(ctx, buddyBTreeFields));
-    }
-
-    // This function in an instance of this index is only used after a bulk load
-    // is successful
-    // it will therefore add the component to the first list and enter it.
-    @Override
-    public void addDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        if (version == 0) {
-            diskComponents.add(0, c);
-        } else if (version == 1) {
-            secondDiskComponents.add(0, c);
-        }
-    }
-
-    @Override
-    public int getCurrentVersion() {
-        return version;
-    }
-
-    @Override
-    public void setCurrentVersion(int version) {
-        this.version = version;
-    }
-
-    @Override
-    public List<ILSMDiskComponent> getFirstComponentList() {
-        return diskComponents;
-    }
-
-    @Override
-    public List<ILSMDiskComponent> getSecondComponentList() {
-        return secondDiskComponents;
-    }
-
-    @Override
-    public void commitTransaction() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
-        ILSMDiskComponent component = null;
-        if (componentFileRefrences != null) {
-            component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
-                    componentFileRefrences.getDeleteIndexFileReference(),
-                    componentFileRefrences.getBloomFilterFileReference(), false);
-        }
-        ((ExternalIndexHarness) getHarness()).addTransactionComponents(component);
-    }
-
-    @Override
-    public void abortTransaction() throws HyracksDataException {
-        fileManager.deleteTransactionFiles();
-    }
-
-    @Override
-    public void recoverTransaction() throws HyracksDataException {
-        fileManager.recoverTransaction();
-    }
-
-    @Override
-    public boolean isPrimaryIndex() {
-        return false;
-    }
-
-    @Override
-    protected LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
-            ILSMDiskComponent lastComponent) throws HyracksDataException {
-        return null;
-    }
-
-    @Override
-    protected AbstractLSMIndexOperationContext createOpContext(IIndexAccessParameters iap) {
-        return null;
-    }
-
-    @Override
-    protected ILSMIOOperation createFlushOperation(AbstractLSMIndexOperationContext opCtx,
-            LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        return null;
-    }
-
-    @Override
-    protected ILSMIOOperation createMergeOperation(AbstractLSMIndexOperationContext opCtx,
-            LSMComponentFileReferences mergeFileRefs, ILSMIOOperationCallback callback) throws HyracksDataException {
-        return null;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
deleted file mode 100644
index 1309e90654..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
+++ /dev/null
@@ -1,84 +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.hyracks.storage.am.lsm.btree.impls;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.MultiComparator;
-import org.apache.hyracks.util.trace.ITracer;
-
-public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationContext {
-    private MultiComparator bTreeCmp;
-    private MultiComparator buddyBTreeCmp;
-    private final int targetIndexVersion;
-    private LSMBTreeWithBuddyCursorInitialState searchInitialState;
-
-    public ExternalBTreeWithBuddyOpContext(ILSMIndex index, IBinaryComparatorFactory[] btreeCmpFactories,
-            IBinaryComparatorFactory[] buddyBtreeCmpFactories, ISearchOperationCallback searchCallback,
-            int targetIndexVersion, ILSMHarness lsmHarness, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, ITreeIndexFrameFactory buddyBtreeLeafFrameFactory,
-            ITracer tracer) {
-        super(index, null, null, null, searchCallback, null, tracer);
-        this.targetIndexVersion = targetIndexVersion;
-        this.bTreeCmp = MultiComparator.create(btreeCmpFactories);
-        this.buddyBTreeCmp = MultiComparator.create(buddyBtreeCmpFactories);
-        searchInitialState = new LSMBTreeWithBuddyCursorInitialState(btreeInteriorFrameFactory, btreeLeafFrameFactory,
-                buddyBtreeLeafFrameFactory, lsmHarness, MultiComparator.create(btreeCmpFactories),
-                MultiComparator.create(buddyBtreeCmpFactories), NoOpOperationCallback.INSTANCE, null);
-    }
-
-    @Override
-    public void setCurrentMutableComponentId(int currentMutableComponentId) {
-        // Do nothing. this should never be called for disk only indexes
-    }
-
-    public MultiComparator getBTreeMultiComparator() {
-        return bTreeCmp;
-    }
-
-    public MultiComparator getBuddyBTreeMultiComparator() {
-        return buddyBTreeCmp;
-    }
-
-    // This should never be needed for disk only indexes
-    @Override
-    public IExtendedModificationOperationCallback getModificationCallback() {
-        return null;
-    }
-
-    public int getTargetIndexVersion() {
-        return targetIndexVersion;
-    }
-
-    public LSMBTreeWithBuddyCursorInitialState getSearchInitialState() {
-        return searchInitialState;
-    }
-
-    @Override
-    public void destroy() throws HyracksDataException {
-        // No Op
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index d2fbbefc06..eceefc68c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -123,25 +123,6 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
         this.hasBloomFilter = hasBloomFilter;
     }
 
-    // Without memory components
-    public LSMBTree(IIOManager ioManager, ITreeIndexFrameFactory insertLeafFrameFactory,
-            ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache bufferCache, ILSMIndexFileManager fileManager,
-            ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
-            double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean needKeyDupCheck, boolean durable, ITracer tracer) throws HyracksDataException {
-        super(ioManager, bufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, bulkLoadComponentFactory, durable,
-                tracer);
-        this.insertLeafFrameFactory = insertLeafFrameFactory;
-        this.deleteLeafFrameFactory = deleteLeafFrameFactory;
-        this.cmpFactories = cmpFactories;
-        this.needKeyDupCheck = needKeyDupCheck;
-        this.hasBloomFilter = true;
-        this.updateAware = false;
-    }
-
     @Override
     public boolean isPrimaryIndex() {
         return needKeyDupCheck;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index 018c0d3dee..c3ca3d5bde 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -30,21 +30,15 @@ import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
-import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
-import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponentFactory;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyDiskComponentFactory;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyFileManager;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
@@ -56,7 +50,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.frames.LSMComponentFilterFrameFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.ComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.impls.DiskBTreeFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
@@ -133,112 +126,4 @@ public class LSMBTreeUtil {
                 cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
                 needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer);
     }
-
-    public static ExternalBTree createExternalBTree(IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean durable, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
-        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
-        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, true, false, nullTypeTraits, nullIntrospector);
-        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
-                cmpFactories.length, false, nullTypeTraits, nullIntrospector);
-        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
-        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
-        ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
-        // This is the tuple writer that can do both inserts and deletes
-        LSMBTreeTupleWriterFactory transactionTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
-        // This is the leaf frame factory for transaction components since it
-        // can be used for both inserts and deletes
-        ITreeIndexFrameFactory transactionLeafFrameFactory =
-                new BTreeNSMLeafFrameFactory(transactionTupleWriterFactory);
-        TreeIndexFactory<DiskBTree> diskBTreeFactory =
-                new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
-                        copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
-        TreeIndexFactory<DiskBTree> bulkLoadBTreeFactory = new DiskBTreeFactory(ioManager, diskBufferCache,
-                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-        // This is the component factory for transactions
-        TreeIndexFactory<DiskBTree> transactionBTreeFactory =
-                new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
-                        transactionLeafFrameFactory, cmpFactories, typeTraits.length);
-        //TODO remove BloomFilter from external dataset's secondary LSMBTree index
-        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, true);
-        ILSMDiskComponentFactory componentFactory =
-                new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
-        ILSMDiskComponentFactory bulkLoadComponentFactory =
-                new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
-        ILSMDiskComponentFactory transactionComponentFactory =
-                new LSMBTreeWithBloomFilterDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
-        // the disk only index uses an empty ArrayList for virtual buffer caches
-        return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
-                diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
-                transactionComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, durable, tracer);
-    }
-
-    public static ExternalBTreeWithBuddy createExternalBTreeWithBuddy(IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] buddyBTreeFields, boolean durable,
-            IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) throws HyracksDataException {
-        ITypeTraits[] buddyBtreeTypeTraits = new ITypeTraits[buddyBTreeFields.length];
-        IBinaryComparatorFactory[] buddyBtreeCmpFactories = new IBinaryComparatorFactory[buddyBTreeFields.length];
-        for (int i = 0; i < buddyBtreeTypeTraits.length; i++) {
-            buddyBtreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
-            buddyBtreeCmpFactories[i] = cmpFactories[buddyBTreeFields[i]];
-        }
-        BTreeTypeAwareTupleWriterFactory buddyBtreeTupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(buddyBtreeTypeTraits, false, nullTypeTraits, nullIntrospector);
-        ITreeIndexFrameFactory buddyBtreeInteriorFrameFactory =
-                new BTreeNSMInteriorFrameFactory(buddyBtreeTupleWriterFactory);
-        ITreeIndexFrameFactory buddyBtreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(buddyBtreeTupleWriterFactory);
-
-        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
-        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
-                cmpFactories.length, false, nullTypeTraits, nullIntrospector);
-        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
-        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
-        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, freePageManagerFactory,
-                interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
-
-        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(ioManager, diskBufferCache,
-                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-
-        int[] bloomFilterKeyFields = new int[buddyBtreeCmpFactories.length];
-        for (int i = 0; i < buddyBtreeCmpFactories.length; i++) {
-            bloomFilterKeyFields[i] = i;
-        }
-        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-
-        // buddy b-tree factory
-        TreeIndexFactory<BTree> diskBuddyBTreeFactory =
-                new BTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, buddyBtreeInteriorFrameFactory,
-                        buddyBtreeLeafFrameFactory, buddyBtreeCmpFactories, buddyBtreeTypeTraits.length);
-
-        ILSMIndexFileManager fileNameManager =
-                new LSMBTreeWithBuddyFileManager(ioManager, file, diskBTreeFactory, diskBuddyBTreeFactory);
-
-        ILSMDiskComponentFactory componentFactory = new LSMBTreeWithBuddyDiskComponentFactory(diskBTreeFactory,
-                diskBuddyBTreeFactory, bloomFilterFactory, null);
-        ILSMDiskComponentFactory bulkLoadComponentFactory = new LSMBTreeWithBuddyDiskComponentFactory(
-                bulkLoadBTreeFactory, diskBuddyBTreeFactory, bloomFilterFactory, null);
-
-        // the disk only index uses an empty ArrayList for virtual buffer caches
-        return new ExternalBTreeWithBuddy(ioManager, interiorFrameFactory, insertLeafFrameFactory,
-                buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
-                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackFactory, pageWriteCallbackFactory, cmpFactories, buddyBtreeCmpFactories, buddyBTreeFields,
-                durable, tracer);
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index bf93dc0551..2745b44b3a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -152,42 +152,6 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         }
     }
 
-    // The constructor used by external indexes
-    public AbstractLSMIndex(IIOManager ioManager, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILSMDiskComponentFactory componentFactory,
-            ILSMDiskComponentFactory bulkLoadComponentFactory, boolean durable, ITracer tracer)
-            throws HyracksDataException {
-        this.ioManager = ioManager;
-        this.diskBufferCache = diskBufferCache;
-        this.fileManager = fileManager;
-        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
-        this.pageWriteCallbackFactory = pageWriteCallbackFactory;
-        this.componentFactory = componentFactory;
-        this.bulkLoadComponentFactory = bulkLoadComponentFactory;
-        this.durable = durable;
-        this.tracer = tracer;
-        lsmHarness = new ExternalIndexHarness(this, ioScheduler, mergePolicy, opTracker,
-                diskBufferCache.isReplicationEnabled());
-        isActive = false;
-        diskComponents = new ArrayList<>();
-        this.inactiveDiskComponents = new ArrayList<>();
-        this.inactiveMemoryComponents = new ArrayList<>();
-        // Memory related objects are nulled
-        virtualBufferCaches = null;
-        memoryComponents = null;
-        currentMutableComponentId = null;
-        flushRequests = null;
-        filterHelper = null;
-        filterFrameFactory = null;
-        filterManager = null;
-        treeFields = null;
-        filterFields = null;
-        fileManager.initLastUsedSeq(ioOpCallback.getLastValidSequence());
-    }
-
     @Override
     public synchronized void create() throws HyracksDataException {
         if (isActive) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
deleted file mode 100644
index 854e5416b4..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ /dev/null
@@ -1,332 +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.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.util.trace.ITracer;
-
-public class ExternalIndexHarness extends LSMHarness {
-    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, boolean replicationEnabled) {
-        super(lsmIndex, ioScheduler, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
-    }
-
-    @Override
-    protected boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
-            boolean isTryOperation) throws HyracksDataException {
-        validateOperationEnterComponentsState(ctx);
-        synchronized (opTracker) {
-            while (true) {
-                lsmIndex.getOperationalComponents(ctx);
-                // Before entering the components, prune those corner cases that indeed should not proceed.
-                switch (opType) {
-                    case MERGE:
-                        if (ctx.getComponentHolder().size() < 2) {
-                            // There is only a single component. There is nothing to merge.
-                            return false;
-                        }
-                    default:
-                        break;
-                }
-                if (enterComponents(ctx, opType)) {
-                    return true;
-                } else if (isTryOperation) {
-                    return false;
-                }
-            }
-        }
-    }
-
-    @Override
-    protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType)
-            throws HyracksDataException {
-        validateOperationEnterComponentsState(ctx);
-        List<ILSMComponent> components = ctx.getComponentHolder();
-        int numEntered = 0;
-        boolean entranceSuccessful = false;
-        try {
-            for (ILSMComponent c : components) {
-                if (!c.threadEnter(opType, false)) {
-                    break;
-                }
-                numEntered++;
-            }
-            entranceSuccessful = numEntered == components.size();
-        } finally {
-            if (!entranceSuccessful) {
-                for (ILSMComponent c : components) {
-                    if (numEntered == 0) {
-                        break;
-                    }
-                    c.threadExit(opType, true, false);
-                    numEntered--;
-                }
-                return false;
-            }
-            ctx.setAccessingComponents(true);
-        }
-        opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
-        return true;
-    }
-
-    private void exitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, ILSMDiskComponent newComponent,
-            boolean failedOperation) throws HyracksDataException {
-        /**
-         * FLUSH and MERGE operations should always exit the components
-         * to notify waiting threads.
-         */
-        if (!ctx.isAccessingComponents() && opType != LSMOperationType.FLUSH && opType != LSMOperationType.MERGE) {
-            return;
-        }
-        synchronized (opTracker) {
-            try {
-                // First check if there is any action that is needed to be taken based on the state of each component.
-                for (ILSMComponent c : ctx.getComponentHolder()) {
-                    c.threadExit(opType, failedOperation, false);
-                    switch (c.getState()) {
-                        case INACTIVE:
-                            if (replicationEnabled) {
-                                componentsToBeReplicated.clear();
-                                componentsToBeReplicated.add((ILSMDiskComponent) c);
-                                lsmIndex.scheduleReplication(null, componentsToBeReplicated,
-                                        ReplicationOperation.DELETE, opType);
-                            }
-                            ((ILSMDiskComponent) c).deactivateAndDestroy();
-                            break;
-                        default:
-                            break;
-                    }
-                }
-                ctx.setAccessingComponents(false);
-                // Then, perform any action that is needed to be taken based on the operation type.
-                switch (opType) {
-                    case MERGE:
-                        // newComponent is null if the merge op. was not performed.
-                        if (newComponent != null) {
-                            beforeSubsumeMergedComponents(newComponent, ctx.getComponentHolder());
-                            lsmIndex.subsumeMergedComponents(newComponent, ctx.getComponentHolder());
-                            if (replicationEnabled) {
-                                componentsToBeReplicated.clear();
-                                componentsToBeReplicated.add(newComponent);
-                                triggerReplication(componentsToBeReplicated, opType);
-                            }
-                            mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
-                        }
-                        break;
-                    default:
-                        break;
-                }
-            } finally {
-                opTracker.afterOperation(lsmIndex, opType, ctx.getSearchOperationCallback(),
-                        ctx.getModificationCallback());
-            }
-        }
-    }
-
-    @Override
-    public void forceModify(ILSMIndexOperationContext ctx, ITupleReference tuple) throws HyracksDataException {
-        throw HyracksDataException.create(ErrorCode.MODIFY_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
-    }
-
-    @Override
-    public boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
-            throws HyracksDataException {
-        throw HyracksDataException.create(ErrorCode.MODIFY_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
-    }
-
-    @Override
-    public void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred)
-            throws HyracksDataException {
-        LSMOperationType opType = LSMOperationType.SEARCH;
-        getAndEnterComponents(ctx, opType, false);
-        try {
-            lsmIndex.search(ctx, cursor, pred);
-        } catch (Exception e) {
-            exitComponents(ctx, opType, null, true);
-            throw e;
-        }
-    }
-
-    @Override
-    public void endSearch(ILSMIndexOperationContext ctx) throws HyracksDataException {
-        if (ctx.getOperation() == IndexOperation.SEARCH) {
-            try {
-                exitComponents(ctx, LSMOperationType.SEARCH, null, false);
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
-        }
-    }
-
-    @SuppressWarnings("squid:S1181")
-    @Override
-    public void addBulkLoadedComponent(ILSMIOOperation ioOperation) throws HyracksDataException {
-        ILSMDiskComponent c = ioOperation.getNewComponent();
-        try {
-            c.markAsValid(lsmIndex.isDurable(), ioOperation);
-        } catch (Throwable th) {
-            ioOperation.setFailure(th);
-        }
-        if (ioOperation.hasFailed()) {
-            throw HyracksDataException.create(ioOperation.getFailure());
-        }
-        synchronized (opTracker) {
-            lsmIndex.addDiskComponent(c);
-            if (replicationEnabled) {
-                componentsToBeReplicated.clear();
-                componentsToBeReplicated.add(c);
-                triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
-            }
-            // Enter the component
-            enterComponent(c);
-            mergePolicy.diskComponentAdded(lsmIndex, false);
-        }
-    }
-
-    // Three differences from  addBulkLoadedComponent
-    // 1. this needs synchronization since others might be accessing the index (specifically merge operations that might change the lists of components)
-    // 2. the actions taken by the index itself are different
-    // 3. the component has already been marked valid by the bulk update operation
-    public void addTransactionComponents(ILSMDiskComponent newComponent) throws HyracksDataException {
-        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
-        synchronized (opTracker) {
-            List<ILSMDiskComponent> newerList;
-            List<ILSMDiskComponent> olderList;
-            if (index.getCurrentVersion() == 0) {
-                newerList = index.getFirstComponentList();
-                olderList = index.getSecondComponentList();
-            } else {
-                newerList = index.getSecondComponentList();
-                olderList = index.getFirstComponentList();
-            }
-            // Exit components in old version of the index so they are ready to be
-            // deleted if they are not needed anymore
-            for (ILSMDiskComponent c : olderList) {
-                exitComponent(c);
-            }
-            // Enter components in the newer list
-            for (ILSMDiskComponent c : newerList) {
-                enterComponent(c);
-            }
-            if (newComponent != null) {
-                // Enter new component
-                enterComponent(newComponent);
-            }
-            index.commitTransactionDiskComponent(newComponent);
-            mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
-        }
-    }
-
-    @Override
-    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
-        return NoOpIoOperation.INSTANCE;
-    }
-
-    @Override
-    public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ILSMOperationTracker getOperationTracker() {
-        return opTracker;
-    }
-
-    public void beforeSubsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
-        // check if merge will affect the first list
-        if (index.getFirstComponentList().containsAll(mergedComponents)) {
-            // exit un-needed components
-            for (ILSMComponent c : mergedComponents) {
-                exitComponent((ILSMDiskComponent) c);
-            }
-            // enter new component
-            enterComponent(newComponent);
-        }
-        // check if merge will affect the second list
-        if (index.getSecondComponentList().containsAll(mergedComponents)) {
-            // exit un-needed components
-            for (ILSMComponent c : mergedComponents) {
-                exitComponent((ILSMDiskComponent) c);
-            }
-            // enter new component
-            enterComponent(newComponent);
-        }
-    }
-
-    // The two methods: enterComponent and exitComponent are used to control
-    // when components are to be deleted from disk
-    private void enterComponent(ILSMComponent diskComponent) throws HyracksDataException {
-        diskComponent.threadEnter(LSMOperationType.SEARCH, false);
-    }
-
-    private void exitComponent(ILSMDiskComponent diskComponent) throws HyracksDataException {
-        diskComponent.threadExit(LSMOperationType.SEARCH, false, false);
-        if (diskComponent.getState() == ILSMComponent.ComponentState.INACTIVE) {
-            if (replicationEnabled) {
-                componentsToBeReplicated.clear();
-                componentsToBeReplicated.add(diskComponent);
-                lsmIndex.scheduleReplication(null, componentsToBeReplicated, ReplicationOperation.DELETE, null);
-            }
-            diskComponent.deactivateAndDestroy();
-        }
-    }
-
-    public void indexFirstTimeActivated() throws HyracksDataException {
-        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
-        // Enter disk components <-- To avoid deleting them when they are
-        // still needed-->
-        for (ILSMComponent c : index.getFirstComponentList()) {
-            enterComponent(c);
-        }
-        for (ILSMComponent c : index.getSecondComponentList()) {
-            enterComponent(c);
-        }
-    }
-
-    public void indexClear() throws HyracksDataException {
-        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
-        for (ILSMDiskComponent c : index.getFirstComponentList()) {
-            exitComponent(c);
-        }
-        for (ILSMDiskComponent c : index.getSecondComponentList()) {
-            exitComponent(c);
-        }
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
deleted file mode 100644
index e6d50d0ad6..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.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.hyracks.storage.am.lsm.rtree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.io.IJsonSerializable;
-import org.apache.hyracks.api.io.IPersistedResourceRegistry;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
-import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.IStorageManager;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-/**
- * The local resource class for disk only lsm r-tree
- */
-public class ExternalRTreeLocalResource extends LSMRTreeLocalResource {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalRTreeLocalResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) {
-        super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
-                opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
-    }
-
-    private ExternalRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
-            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields,
-            int[] buddyBTreeFields, boolean isPointMBR, double bloomFilterFalsePositiveRate)
-            throws HyracksDataException {
-        super(registry, json, btreeCmpFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory,
-                rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
-    }
-
-    @Override
-    public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
-        IIOManager ioManager = ncServiceCtx.getIoManager();
-        FileReference fileRef = ioManager.resolve(path);
-        ioOpCallbackFactory.initialize(ncServiceCtx, this);
-        pageWriteCallbackFactory.initialize(ncServiceCtx, this);
-        return LSMRTreeUtils.createExternalRTree(ioManager, fileRef, storageManager.getBufferCache(ncServiceCtx),
-                typeTraits, cmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                bloomFilterFalsePositiveRate, mergePolicyFactory.createMergePolicy(mergePolicyProperties, ncServiceCtx),
-                opTrackerProvider.getOperationTracker(ncServiceCtx, this),
-                ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
-                linearizeCmpFactory, buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory,
-                ncServiceCtx.getTracer(), nullTypeTraits, nullIntrospector);
-
-    }
-
-    @Override
-    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
-        final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
-        super.appendToJson(jsonObject, registry);
-        return jsonObject;
-    }
-
-    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
-            throws HyracksDataException {
-        LSMRTreeLocalResource lsmRTree = (LSMRTreeLocalResource) LSMRTreeLocalResource.fromJson(registry, json);
-        return new ExternalRTreeLocalResource(registry, json, lsmRTree.btreeCmpFactories,
-                lsmRTree.valueProviderFactories, lsmRTree.rtreePolicyType, lsmRTree.linearizeCmpFactory,
-                lsmRTree.rtreeFields, lsmRTree.buddyBTreeFields, lsmRTree.isPointMBR,
-                lsmRTree.bloomFilterFalsePositiveRate);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
deleted file mode 100644
index ca8caecb66..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
+++ /dev/null
@@ -1,72 +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.hyracks.storage.am.lsm.rtree.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.api.INullIntrospector;
-import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
-import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import org.apache.hyracks.storage.common.IStorageManager;
-
-public class ExternalRTreeLocalResourceFactory extends LSMRTreeLocalResourceFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public ExternalRTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) {
-        super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
-                opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
-                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
-    }
-
-    @Override
-    public LsmResource createResource(FileReference fileRef) {
-        return new ExternalRTreeLocalResource(fileRef.getRelativePath(), storageManager, typeTraits, cmpFactories,
-                filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
-                pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
-                mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate,
-                nullTypeTraits, nullIntrospector);
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index 1b847a0e1d..e95ef67fb1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -117,34 +117,6 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
         this.isPointMBR = isPointMBR;
     }
 
-    /*
-     * For External indexes with no memory components
-     */
-    public AbstractLSMRTree(IIOManager ioManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
-            ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
-            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean durable, boolean isPointMBR, ITracer tracer)
-            throws HyracksDataException {
-        super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, componentFactory, durable,
-                tracer);
-        this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
-        this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
-        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
-        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
-        this.btreeCmpFactories = btreeCmpFactories;
-        this.rtreeCmpFactories = rtreeCmpFactories;
-        this.linearizer = linearizer;
-        this.comparatorFields = comparatorFields;
-        this.linearizerArray = linearizerArray;
-        this.isPointMBR = isPointMBR;
-    }
-
     @Override
     public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
             throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
deleted file mode 100644
index 5557bbe37c..0000000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ /dev/null
@@ -1,656 +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.hyracks.storage.am.lsm.rtree.impls;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
-import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
-import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
-import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.IIndexCursorStats;
-import org.apache.hyracks.storage.common.ISearchOperationCallback;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.IndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.util.trace.ITracer;
-
-/**
- * This is an lsm r-tree that does not have memory component and is modified
- * only by bulk loading and addition of disk components as of this point, it is
- * intended for use with external dataset indexes only.
- */
-public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
-
-    // A second disk component list that will be used when a transaction is
-    // committed and will be seen by subsequent accessors
-    private final List<ILSMDiskComponent> secondDiskComponents;
-    // A pointer that points to the current most recent list (either
-    // diskComponents = 0, or secondDiskComponents = 1). It starts with -1 to
-    // indicate first time activation
-    private int version = 0;
-    private final int fieldCount;
-
-    public ExternalRTree(IIOManager ioManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
-            ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
-            ILSMIndexFileManager fileNameManager, ILSMDiskComponentFactory componentFactory,
-            double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
-            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            int[] buddyBTreeFields, boolean durable, boolean isPointMBR, ITracer tracer) throws HyracksDataException {
-        super(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
-                btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bloomFilterFalsePositiveRate,
-                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, buddyBTreeFields, durable,
-                isPointMBR, tracer);
-        this.secondDiskComponents = new LinkedList<>();
-        this.fieldCount = fieldCount;
-    }
-
-    @Override
-    public ExternalIndexHarness getHarness() {
-        return (ExternalIndexHarness) super.getHarness();
-    }
-
-    // The subsume merged components is overridden to account for:
-    // 1. the number of readers of components
-    // 2. maintaining two versions of the index
-    @Override
-    public void subsumeMergedComponents(ILSMDiskComponent newComponent, List<ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        // determine which list is the new one
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-        }
-        // check if merge will affect the older list
-        if (olderList.containsAll(mergedComponents)) {
-            int swapIndex = olderList.indexOf(mergedComponents.get(0));
-            olderList.removeAll(mergedComponents);
-            olderList.add(swapIndex, newComponent);
-        }
-        // The new list will always have all the merged components
-        int swapIndex = newerList.indexOf(mergedComponents.get(0));
-        newerList.removeAll(mergedComponents);
-        newerList.add(swapIndex, newComponent);
-    }
-
-    // This method is used by the merge policy when it needs to check if a merge
-    // is needed.
-    // It only needs to return the newer list
-    @Override
-    public List<ILSMDiskComponent> getDiskComponents() {
-        if (version == 0) {
-            return diskComponents;
-        } else {
-            return secondDiskComponents;
-        }
-    }
-
-    // This function should only be used when a transaction fail. it doesn't
-    // take any parameters since there can only be
-    // a single transaction and hence a single transaction component on disk
-    public void deleteTransactionComponent() throws HyracksDataException {
-        fileManager.deleteTransactionFiles();
-    }
-
-    // This function in an instance of this index is only used after a bulk load
-    // is successful
-    // it will therefore add the component to the first list and enter it.
-    @Override
-    public void addDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        if (version == 0) {
-            diskComponents.add(0, c);
-        } else if (version == 1) {
-            secondDiskComponents.add(0, c);
-        }
-    }
-
-    // This function is used when a new component is to be committed.
-    @Override
-    public void commitTransactionDiskComponent(ILSMDiskComponent newComponent) throws HyracksDataException {
-
-        // determine which list is the new one and flip the pointer
-        List<ILSMDiskComponent> newerList;
-        List<ILSMDiskComponent> olderList;
-        if (version == 0) {
-            newerList = diskComponents;
-            olderList = secondDiskComponents;
-            version = 1;
-        } else {
-            newerList = secondDiskComponents;
-            olderList = diskComponents;
-            version = 0;
-        }
-        // Remove components from list
-        olderList.clear();
-        // Add components
-        olderList.addAll(newerList);
-        if (newComponent != null) {
-            // Add it to the list
-            olderList.add(0, newComponent);
-        }
-    }
-
-    @Override
-    public synchronized void activate() throws HyracksDataException {
-        if (isActive) {
-            throw new HyracksDataException("Failed to activate the index since it is already activated.");
-        }
-
-        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
-            //First time activation
-            List<LSMComponentFileReferences> validFileReferences;
-            validFileReferences = fileManager.cleanupAndGetValidFiles();
-            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-                ILSMDiskComponent component;
-                component =
-                        createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(),
-                                lsmComonentFileReference.getDeleteIndexFileReference(),
-                                lsmComonentFileReference.getBloomFilterFileReference(), false);
-                diskComponents.add(component);
-                secondDiskComponents.add(component);
-            }
-            getHarness().indexFirstTimeActivated();
-        } else {
-            // This index has been opened before or is brand new with no components
-            // components. It should also maintain the version pointer
-            for (ILSMComponent c : diskComponents) {
-                LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-                component.activate(false);
-            }
... 644 lines suppressed ...